From 8f14233166d33836db3ecb973ab3ac0aacb568f5 Mon Sep 17 00:00:00 2001 From: SangBin Cho Date: Mon, 1 Feb 2021 22:06:59 -0800 Subject: [PATCH 1/5] in progress. --- python/ray/internal/internal_api.py | 11 ++- python/ray/scripts/scripts.py | 11 ++- python/ray/tests/test_object_spilling.py | 88 ++++++++++++++++++------ src/ray/core_worker/core_worker.cc | 18 +++-- src/ray/core_worker/core_worker.h | 13 ++++ src/ray/protobuf/common.proto | 2 + src/ray/protobuf/node_manager.proto | 2 + src/ray/raylet/node_manager.cc | 7 +- src/ray/raylet/node_manager.h | 4 ++ 9 files changed, 123 insertions(+), 33 deletions(-) diff --git a/python/ray/internal/internal_api.py b/python/ray/internal/internal_api.py index 67c1a9275f37..28252f9dbb6b 100644 --- a/python/ray/internal/internal_api.py +++ b/python/ray/internal/internal_api.py @@ -13,7 +13,9 @@ def global_gc(): worker.core_worker.global_gc() -def memory_summary(node_manager_address=None, node_manager_port=None): +def memory_summary(node_manager_address=None, + node_manager_port=None, + stats_only=False): """Returns a formatted string describing memory usage in the cluster.""" import grpc @@ -63,6 +65,13 @@ def memory_summary(node_manager_address=None, node_manager_port=None): reply.store_stats.restored_objects_total, int(reply.store_stats.restored_bytes_total / (1024 * 1024) / reply.store_stats.restore_time_total_s))) + if reply.store_stats.referenced_bytes > 0: + store_summary += ( + "Total bytes of plasma objects referenced by ray tasks " + "or ray.get: {} MiB.".format( + int(reply.store_stats.referenced_bytes / (1024 * 1024)))) + if stats_only: + return store_summary return reply.memory_summary + "\n" + store_summary diff --git a/python/ray/scripts/scripts.py b/python/ray/scripts/scripts.py index b61c6939984c..5c3ce1012b17 100644 --- a/python/ray/scripts/scripts.py +++ b/python/ray/scripts/scripts.py @@ -1372,7 +1372,13 @@ def timeline(address): type=str, default=ray_constants.REDIS_DEFAULT_PASSWORD, help="Connect to ray with redis_password.") -def memory(address, redis_password): +@click.option( + "--stats-only", + is_flag=True, + type=bool, + default=False, + help="Connect to ray with redis_password.") +def memory(address, redis_password, stats_only): """Print object references held in a Ray cluster.""" if not address: address = services.get_ray_address_to_use_or_die() @@ -1381,7 +1387,8 @@ def memory(address, redis_password): raylet = state.node_table()[0] print( ray.internal.internal_api.memory_summary(raylet["NodeManagerAddress"], - raylet["NodeManagerPort"])) + raylet["NodeManagerPort"], + stats_only)) @cli.command() diff --git a/python/ray/tests/test_object_spilling.py b/python/ray/tests/test_object_spilling.py index 159e0aaf79b1..7c8f2941271f 100644 --- a/python/ray/tests/test_object_spilling.py +++ b/python/ray/tests/test_object_spilling.py @@ -88,6 +88,27 @@ def is_dir_empty(temp_folder, return num_files == 0 +def assert_no_thrashing(address): + state = ray.state.GlobalState() + state._initialize_global_state(address, + ray.ray_constants.REDIS_DEFAULT_PASSWORD) + raylet = state.node_table()[0] + memory_summary = ray.internal.internal_api.memory_summary( + raylet["NodeManagerAddress"], + raylet["NodeManagerPort"], + stats_only=True) + restored_bytes = 0 + referenced_bytes = 0 + + for line in memory_summary.split("\n"): + if "Restored" in line: + restored_bytes = int(line.split(" ")[1]) + if "referenced" in line: + referenced_bytes = int(line.split(" ")[-2]) + assert referenced_bytes >= restored_bytes, ( + f"referenced: {referenced_bytes}, restored: {restored_bytes}") + + def test_invalid_config_raises_exception(shutdown_only): # Make sure ray.init raises an exception before # it starts processes when invalid object spilling @@ -187,7 +208,7 @@ def test_spilling_not_done_for_pinned_object(object_spilling_config, shutdown_only): # Limit our object store to 75 MiB of memory. object_spilling_config, temp_folder = object_spilling_config - ray.init( + address = ray.init( object_store_memory=75 * 1024 * 1024, _system_config={ "max_io_workers": 4, @@ -203,6 +224,7 @@ def test_spilling_not_done_for_pinned_object(object_spilling_config, ref2 = ray.put(arr) # noqa wait_for_condition(lambda: is_dir_empty(temp_folder)) + assert_no_thrashing(address["redis_address"]) @pytest.mark.skipif( @@ -249,6 +271,7 @@ def depends(arg): # Test passing the spilled object as an arg to another task. ray.get(depends.remote(ref)) + assert_no_thrashing(cluster.address) @pytest.mark.skipif( @@ -256,7 +279,7 @@ def depends(arg): def test_spill_objects_automatically(object_spilling_config, shutdown_only): # Limit our object store to 75 MiB of memory. object_spilling_config, _ = object_spilling_config - ray.init( + address = ray.init( num_cpus=1, object_store_memory=75 * 1024 * 1024, _system_config={ @@ -287,6 +310,7 @@ def test_spill_objects_automatically(object_spilling_config, shutdown_only): solution = solution_buffer[index] sample = ray.get(ref, timeout=0) assert np.array_equal(sample, solution) + assert_no_thrashing(address["redis_address"]) @pytest.mark.skipif( @@ -294,7 +318,7 @@ def test_spill_objects_automatically(object_spilling_config, shutdown_only): def test_spill_stats(object_spilling_config, shutdown_only): # Limit our object store to 75 MiB of memory. object_spilling_config, _ = object_spilling_config - ray.init( + address = ray.init( num_cpus=1, object_store_memory=100 * 1024 * 1024, _system_config={ @@ -319,17 +343,31 @@ def f(): x_id = f.remote() # noqa ray.get(x_id) - s = memory_summary() + s = memory_summary(stats_only=True) assert "Plasma memory usage 50 MiB, 1 objects, 50.0% full" in s, s assert "Spilled 200 MiB, 4 objects" in s, s assert "Restored 150 MiB, 3 objects" in s, s + # Test if referenced bytes are correctly calculated. + obj = ray.put(np.zeros(30 * 1024 * 1024, dtype=np.uint8)) + + @ray.remote + def func_with_ref(obj): + return True + + ray.get(func_with_ref.remote(obj)) + + s = memory_summary(stats_only=True) + # 50MB * 5 references + 30MB used for task execution. + assert "ray.get: 280 MiB." in s, s + assert_no_thrashing(address["redis_address"]) + @pytest.mark.skipif( platform.system() == "Windows", reason="Failing on Windows.") def test_spill_during_get(object_spilling_config, shutdown_only): object_spilling_config, _ = object_spilling_config - ray.init( + address = ray.init( num_cpus=4, object_store_memory=100 * 1024 * 1024, _system_config={ @@ -355,6 +393,7 @@ def f(): # objects are being created. for x in ids: print(ray.get(x).shape) + assert_no_thrashing(address["redis_address"]) @pytest.mark.skipif( @@ -362,7 +401,7 @@ def f(): def test_spill_deadlock(object_spilling_config, shutdown_only): object_spilling_config, _ = object_spilling_config # Limit our object store to 75 MiB of memory. - ray.init( + address = ray.init( object_store_memory=75 * 1024 * 1024, _system_config={ "max_io_workers": 1, @@ -386,6 +425,7 @@ def test_spill_deadlock(object_spilling_config, shutdown_only): ref = random.choice(replay_buffer) sample = ray.get(ref, timeout=0) assert np.array_equal(sample, arr) + assert_no_thrashing(address["redis_address"]) @pytest.mark.skipif( @@ -394,7 +434,7 @@ def test_delete_objects(object_spilling_config, shutdown_only): # Limit our object store to 75 MiB of memory. object_spilling_config, temp_folder = object_spilling_config - ray.init( + address = ray.init( object_store_memory=75 * 1024 * 1024, _system_config={ "max_io_workers": 1, @@ -417,6 +457,7 @@ def test_delete_objects(object_spilling_config, shutdown_only): del replay_buffer del ref wait_for_condition(lambda: is_dir_empty(temp_folder)) + assert_no_thrashing(address["redis_address"]) @pytest.mark.skipif( @@ -426,7 +467,7 @@ def test_delete_objects_delete_while_creating(object_spilling_config, # Limit our object store to 75 MiB of memory. object_spilling_config, temp_folder = object_spilling_config - ray.init( + address = ray.init( object_store_memory=75 * 1024 * 1024, _system_config={ "max_io_workers": 4, @@ -457,6 +498,7 @@ def test_delete_objects_delete_while_creating(object_spilling_config, del replay_buffer del ref wait_for_condition(lambda: is_dir_empty(temp_folder)) + assert_no_thrashing(address["redis_address"]) @pytest.mark.skipif( @@ -466,7 +508,7 @@ def test_delete_objects_on_worker_failure(object_spilling_config, # Limit our object store to 75 MiB of memory. object_spilling_config, temp_folder = object_spilling_config - ray.init( + address = ray.init( object_store_memory=75 * 1024 * 1024, _system_config={ "max_io_workers": 4, @@ -518,6 +560,7 @@ def wait_until_actor_dead(): # After all, make sure all objects are deleted upon worker failures. wait_for_condition(lambda: is_dir_empty(temp_folder)) + assert_no_thrashing(address["redis_address"]) @pytest.mark.skipif( @@ -539,10 +582,11 @@ def test_delete_objects_multi_node(multi_node_object_spilling_config, "object_store_full_delay_ms": 100, "object_spilling_config": object_spilling_config, }) + ray.init(address=cluster.address) # Add 2 worker nodes. for _ in range(2): cluster.add_node(num_cpus=1, object_store_memory=75 * 1024 * 1024) - ray.init(address=cluster.address) + cluster.wait_for_nodes() arr = np.random.rand(1024 * 1024) # 8 MB data @@ -565,9 +609,9 @@ def create_objects(self): self.replay_buffer.pop() # Do random sampling. - for _ in range(200): + for _ in range(50): ref = random.choice(self.replay_buffer) - sample = ray.get(ref, timeout=0) + sample = ray.get(ref, timeout=10) assert np.array_equal(sample, arr) actors = [Actor.remote() for _ in range(3)] @@ -586,6 +630,11 @@ def wait_until_actor_dead(actor): wait_for_condition(lambda: wait_until_actor_dead(actor)) # The multi node deletion should work. wait_for_condition(lambda: is_dir_empty(temp_folder)) + # NOTE: Currently, this thrashing calculation logic could be wrong if the + # core worker dies before it reports the referenced bytes. + # TODO(sang): Re-enable after finding a more + # robust thrashing tracking method. + # assert_no_thrashing(cluster.address) @pytest.mark.skipif(platform.system() == "Windows", reason="Flaky on Windows.") @@ -593,7 +642,7 @@ def test_fusion_objects(object_spilling_config, shutdown_only): # Limit our object store to 75 MiB of memory. object_spilling_config, temp_folder = object_spilling_config min_spilling_size = 10 * 1024 * 1024 - ray.init( + address = ray.init( object_store_memory=75 * 1024 * 1024, _system_config={ "max_io_workers": 3, @@ -637,12 +686,13 @@ def test_fusion_objects(object_spilling_config, shutdown_only): if file_size >= min_spilling_size: is_test_passing = True assert is_test_passing + assert_no_thrashing(address["redis_address"]) # https://github.com/ray-project/ray/issues/12912 def do_test_release_resource(object_spilling_config, expect_released): object_spilling_config, temp_folder = object_spilling_config - ray.init( + address = ray.init( num_cpus=1, object_store_memory=75 * 1024 * 1024, _system_config={ @@ -674,6 +724,7 @@ def f(dep): assert ready else: assert not ready + assert_no_thrashing(address["redis_address"]) @pytest.mark.skipif( @@ -745,6 +796,7 @@ def allocate(*args): # spilling. tasks = [foo.remote(*task_args) for task_args in args] ray.get(tasks) + assert_no_thrashing(cluster.address) @pytest.mark.skipif( @@ -801,14 +853,6 @@ def test_file_deleted_when_driver_exits(tmp_path, shutdown_only): driver.format(temp_dir=str(temp_folder), signum=2))) wait_for_condition(lambda: is_dir_empty(temp_folder, append_path="")) - # Q: Looks like Sigterm doesn't work with Ray? - # print("Sending sigterm...") - # # Run a driver with sigterm. - # with pytest.raises(subprocess.CalledProcessError): - # print(run_string_as_driver( - # driver.format(temp_dir=str(temp_folder), signum=15))) - # wait_for_condition(is_dir_empty, timeout=1000) - if __name__ == "__main__": sys.exit(pytest.main(["-sv", __file__])) diff --git a/src/ray/core_worker/core_worker.cc b/src/ray/core_worker/core_worker.cc index 1961406d8a8a..31788fbfa0b4 100644 --- a/src/ray/core_worker/core_worker.cc +++ b/src/ray/core_worker/core_worker.cc @@ -566,6 +566,8 @@ CoreWorker::CoreWorker(const CoreWorkerOptions &options, const WorkerID &worker_ // NOTE: This also marks the worker as available in Raylet. We do this at the // very end in case there is a problem during construction. RAY_CHECK_OK(local_raylet_client_->AnnounceWorkerPort(core_worker_server_->GetPort())); + // Used to detect if the object is in the plasma store. + max_direct_call_object_size_ = RayConfig::instance().max_direct_call_object_size(); } void CoreWorker::Shutdown() { @@ -880,8 +882,7 @@ Status CoreWorker::Put(const RayObject &object, bool object_exists; if (options_.is_local_mode || (RayConfig::instance().put_small_object_in_memory_store() && - static_cast(object.GetSize()) < - RayConfig::instance().max_direct_call_object_size())) { + static_cast(object.GetSize()) < max_direct_call_object_size_)) { RAY_LOG(DEBUG) << "Put " << object_id << " in memory store"; RAY_CHECK(memory_store_->Put(object, object_id)); return Status::OK(); @@ -922,8 +923,7 @@ Status CoreWorker::CreateOwned(const std::shared_ptr &metadata, NodeID::FromBinary(rpc_address_.raylet_id())); if (options_.is_local_mode || (RayConfig::instance().put_small_object_in_memory_store() && - static_cast(data_size) < - RayConfig::instance().max_direct_call_object_size())) { + static_cast(data_size) < max_direct_call_object_size_)) { *data = std::make_shared(data_size); } else { auto status = @@ -1036,8 +1036,9 @@ Status CoreWorker::Get(const std::vector &ids, const int64_t timeout_m bool missing_result = false; bool will_throw_exception = false; for (size_t i = 0; i < ids.size(); i++) { - auto pair = result_map.find(ids[i]); + const auto pair = result_map.find(ids[i]); if (pair != result_map.end()) { + RecordPlasmaReferencedBytes(pair->second); (*results)[i] = pair->second; RAY_CHECK(!pair->second->IsInPlasmaError()); if (pair->second->IsException()) { @@ -1777,8 +1778,7 @@ Status CoreWorker::AllocateReturnObjects( // Allocate a buffer for the return object. if (options_.is_local_mode || - static_cast(data_sizes[i]) < - RayConfig::instance().max_direct_call_object_size()) { + static_cast(data_sizes[i]) < max_direct_call_object_size_) { data_buffer = std::make_shared(data_sizes[i]); } else { RAY_RETURN_NOT_OK(CreateExisting(metadatas[i], data_sizes[i], object_ids[i], @@ -2024,6 +2024,7 @@ Status CoreWorker::GetAndPinArgsForExecutor(const TaskSpecification &task, } for (const auto &it : result_map) { for (size_t idx : by_ref_indices[it.first]) { + RecordPlasmaReferencedBytes(it.second); args->at(idx) = it.second; } } @@ -2359,6 +2360,9 @@ void CoreWorker::HandleGetCoreWorkerStats(const rpc::GetCoreWorkerStatsRequest & stats->set_worker_id(worker_context_.GetWorkerID().Binary()); stats->set_actor_id(actor_id_.Binary()); stats->set_worker_type(worker_context_.GetWorkerType()); + stats->set_referenced_bytes(total_referenced_bytes_); + // Reset this value so that it can be correctly accumulated to the raylet. + total_referenced_bytes_ = 0; auto used_resources_map = stats->mutable_used_resources(); for (auto const &it : *resource_ids_) { rpc::ResourceAllocations allocations; diff --git a/src/ray/core_worker/core_worker.h b/src/ray/core_worker/core_worker.h index 89331b5ce10f..a51310471e5d 100644 --- a/src/ray/core_worker/core_worker.h +++ b/src/ray/core_worker/core_worker.h @@ -1093,6 +1093,15 @@ class CoreWorker : public rpc::CoreWorkerServiceHandler { return call_site; } + inline void RecordPlasmaReferencedBytes(const std::shared_ptr &object) { + // Record the referenced bytes if the object is in the plasma store. + // It is used to detect the thrashing. + const auto object_size = object->GetSize(); + if (object_size >= max_direct_call_object_size_) { + total_referenced_bytes_ += object_size; + } + } + /// Shared state of the worker. Includes process-level and thread-level state. /// TODO(edoakes): we should move process-level state into this class and make /// this a ThreadContext. @@ -1255,6 +1264,10 @@ class CoreWorker : public rpc::CoreWorkerServiceHandler { /// Whether we are shutting down and not running further tasks. bool exiting_ = false; + int64_t total_referenced_bytes_ = 0; + + int64_t max_direct_call_object_size_; + friend class CoreWorkerTest; }; diff --git a/src/ray/protobuf/common.proto b/src/ray/protobuf/common.proto index 844f44bea723..de3c6d6a2138 100644 --- a/src/ray/protobuf/common.proto +++ b/src/ray/protobuf/common.proto @@ -428,6 +428,8 @@ message CoreWorkerStats { uint32 pid = 22; // The worker type. WorkerType worker_type = 23; + // Total plasma object bytes that are referenced by this core worker. + int64 referenced_bytes = 24; } message MetricPoint { diff --git a/src/ray/protobuf/node_manager.proto b/src/ray/protobuf/node_manager.proto index 386ed988ade3..6c6bafca1d39 100644 --- a/src/ray/protobuf/node_manager.proto +++ b/src/ray/protobuf/node_manager.proto @@ -138,6 +138,8 @@ message ObjectStoreStats { int64 object_store_bytes_avail = 8; // The number of local objects total. int64 num_local_objects = 9; + // The number of plasma object bytes that are referenced by core workers. + int64 referenced_bytes = 10; } message GetNodeStatsReply { diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index cbe287ef721d..01c485ecb02d 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -2509,11 +2509,14 @@ void NodeManager::HandleGetNodeStats(const rpc::GetNodeStatsRequest &node_stats_ request.set_intended_worker_id(worker->WorkerId().Binary()); request.set_include_memory_info(node_stats_request.include_memory_info()); worker->rpc_client()->GetCoreWorkerStats( - request, [reply, worker, all_workers, driver_ids, send_reply_callback]( + request, [this, reply, worker, all_workers, driver_ids, send_reply_callback]( const ray::Status &status, const rpc::GetCoreWorkerStatsReply &r) { + total_referenced_bytes_ += r.core_worker_stats().referenced_bytes(); reply->add_core_workers_stats()->MergeFrom(r.core_worker_stats()); reply->set_num_workers(reply->num_workers() + 1); if (reply->num_workers() == all_workers.size()) { + auto stats = reply->mutable_store_stats(); + stats->set_referenced_bytes(total_referenced_bytes_); send_reply_callback(Status::OK(), nullptr, nullptr); } }); @@ -2545,6 +2548,8 @@ rpc::ObjectStoreStats AccumulateStoreStats( cur_store.object_store_bytes_avail()); store_stats.set_num_local_objects(store_stats.num_local_objects() + cur_store.num_local_objects()); + store_stats.set_referenced_bytes(store_stats.referenced_bytes() + + cur_store.referenced_bytes()); } return store_stats; } diff --git a/src/ray/raylet/node_manager.h b/src/ray/raylet/node_manager.h index 606dc3ac6fa7..cbb4af6ce677 100644 --- a/src/ray/raylet/node_manager.h +++ b/src/ray/raylet/node_manager.h @@ -935,6 +935,10 @@ class NodeManager : public rpc::NodeManagerServiceHandler, /// Managers all bundle-related operations. std::shared_ptr placement_group_resource_manager_; + + /// The total bytes size of plasma objects that are referenced by core workers. + /// It is accumulative. + int64_t total_referenced_bytes_ = 0; }; } // namespace raylet From 5f5b0aa96371981141683e18e2e9dabe3a7b76ce Mon Sep 17 00:00:00 2001 From: SangBin Cho Date: Mon, 1 Feb 2021 23:37:57 -0800 Subject: [PATCH 2/5] Fixed implementation. --- python/ray/tests/test_object_spilling.py | 8 ++--- src/ray/core_worker/core_worker.cc | 5 --- src/ray/core_worker/core_worker.h | 11 ------- .../store_provider/plasma_store_provider.cc | 7 ++-- src/ray/object_manager/object_buffer_pool.cc | 3 +- src/ray/object_manager/object_manager.cc | 3 ++ src/ray/object_manager/plasma/client.cc | 32 +++++++++++-------- src/ray/object_manager/plasma/client.h | 6 ++-- src/ray/object_manager/plasma/plasma.fbs | 2 ++ src/ray/object_manager/plasma/protocol.cc | 9 +++--- src/ray/object_manager/plasma/protocol.h | 6 ++-- src/ray/object_manager/plasma/store.cc | 26 +++++++++++---- src/ray/object_manager/plasma/store.h | 9 +++++- src/ray/object_manager/plasma/store_runner.cc | 2 ++ src/ray/object_manager/plasma/store_runner.h | 2 ++ src/ray/protobuf/common.proto | 2 -- src/ray/raylet/node_manager.cc | 9 +++--- src/ray/raylet/node_manager.h | 4 --- 18 files changed, 81 insertions(+), 65 deletions(-) diff --git a/python/ray/tests/test_object_spilling.py b/python/ray/tests/test_object_spilling.py index 7c8f2941271f..6cebeb0bf373 100644 --- a/python/ray/tests/test_object_spilling.py +++ b/python/ray/tests/test_object_spilling.py @@ -314,7 +314,7 @@ def test_spill_objects_automatically(object_spilling_config, shutdown_only): @pytest.mark.skipif( - platform.system() in ["Darwin", "Windows"], reason="Failing on Windows.") + platform.system() in ["Windows", "Darwin"], reason="Failing on Windows.") def test_spill_stats(object_spilling_config, shutdown_only): # Limit our object store to 75 MiB of memory. object_spilling_config, _ = object_spilling_config @@ -630,11 +630,7 @@ def wait_until_actor_dead(actor): wait_for_condition(lambda: wait_until_actor_dead(actor)) # The multi node deletion should work. wait_for_condition(lambda: is_dir_empty(temp_folder)) - # NOTE: Currently, this thrashing calculation logic could be wrong if the - # core worker dies before it reports the referenced bytes. - # TODO(sang): Re-enable after finding a more - # robust thrashing tracking method. - # assert_no_thrashing(cluster.address) + assert_no_thrashing(cluster.address) @pytest.mark.skipif(platform.system() == "Windows", reason="Flaky on Windows.") diff --git a/src/ray/core_worker/core_worker.cc b/src/ray/core_worker/core_worker.cc index 31788fbfa0b4..3c9b101ceb6f 100644 --- a/src/ray/core_worker/core_worker.cc +++ b/src/ray/core_worker/core_worker.cc @@ -1038,7 +1038,6 @@ Status CoreWorker::Get(const std::vector &ids, const int64_t timeout_m for (size_t i = 0; i < ids.size(); i++) { const auto pair = result_map.find(ids[i]); if (pair != result_map.end()) { - RecordPlasmaReferencedBytes(pair->second); (*results)[i] = pair->second; RAY_CHECK(!pair->second->IsInPlasmaError()); if (pair->second->IsException()) { @@ -2024,7 +2023,6 @@ Status CoreWorker::GetAndPinArgsForExecutor(const TaskSpecification &task, } for (const auto &it : result_map) { for (size_t idx : by_ref_indices[it.first]) { - RecordPlasmaReferencedBytes(it.second); args->at(idx) = it.second; } } @@ -2360,9 +2358,6 @@ void CoreWorker::HandleGetCoreWorkerStats(const rpc::GetCoreWorkerStatsRequest & stats->set_worker_id(worker_context_.GetWorkerID().Binary()); stats->set_actor_id(actor_id_.Binary()); stats->set_worker_type(worker_context_.GetWorkerType()); - stats->set_referenced_bytes(total_referenced_bytes_); - // Reset this value so that it can be correctly accumulated to the raylet. - total_referenced_bytes_ = 0; auto used_resources_map = stats->mutable_used_resources(); for (auto const &it : *resource_ids_) { rpc::ResourceAllocations allocations; diff --git a/src/ray/core_worker/core_worker.h b/src/ray/core_worker/core_worker.h index a51310471e5d..6fa24c29e94e 100644 --- a/src/ray/core_worker/core_worker.h +++ b/src/ray/core_worker/core_worker.h @@ -1093,15 +1093,6 @@ class CoreWorker : public rpc::CoreWorkerServiceHandler { return call_site; } - inline void RecordPlasmaReferencedBytes(const std::shared_ptr &object) { - // Record the referenced bytes if the object is in the plasma store. - // It is used to detect the thrashing. - const auto object_size = object->GetSize(); - if (object_size >= max_direct_call_object_size_) { - total_referenced_bytes_ += object_size; - } - } - /// Shared state of the worker. Includes process-level and thread-level state. /// TODO(edoakes): we should move process-level state into this class and make /// this a ThreadContext. @@ -1264,8 +1255,6 @@ class CoreWorker : public rpc::CoreWorkerServiceHandler { /// Whether we are shutting down and not running further tasks. bool exiting_ = false; - int64_t total_referenced_bytes_ = 0; - int64_t max_direct_call_object_size_; friend class CoreWorkerTest; diff --git a/src/ray/core_worker/store_provider/plasma_store_provider.cc b/src/ray/core_worker/store_provider/plasma_store_provider.cc index b42c4b50941f..f3b5f047c8fc 100644 --- a/src/ray/core_worker/store_provider/plasma_store_provider.cc +++ b/src/ray/core_worker/store_provider/plasma_store_provider.cc @@ -191,7 +191,8 @@ Status CoreWorkerPlasmaStoreProvider::FetchAndGetFromPlasmaStore( std::vector plasma_results; { std::lock_guard guard(store_client_mutex_); - RAY_RETURN_NOT_OK(store_client_.Get(batch_ids, timeout_ms, &plasma_results)); + RAY_RETURN_NOT_OK(store_client_.Get(batch_ids, timeout_ms, &plasma_results, + /*is_from_worker=*/true)); } // Add successfully retrieved objects to the result map and remove them from @@ -231,7 +232,9 @@ Status CoreWorkerPlasmaStoreProvider::GetIfLocal( std::vector plasma_results; { std::lock_guard guard(store_client_mutex_); - RAY_RETURN_NOT_OK(store_client_.Get(object_ids, /*timeout_ms=*/0, &plasma_results)); + // Since this path is used only for spilling, we should set is_from_worker: false. + RAY_RETURN_NOT_OK(store_client_.Get(object_ids, /*timeout_ms=*/0, &plasma_results, + /*is_from_worker=*/false)); } for (size_t i = 0; i < object_ids.size(); i++) { diff --git a/src/ray/object_manager/object_buffer_pool.cc b/src/ray/object_manager/object_buffer_pool.cc index 726a6fefca35..63dabcb419ef 100644 --- a/src/ray/object_manager/object_buffer_pool.cc +++ b/src/ray/object_manager/object_buffer_pool.cc @@ -57,7 +57,8 @@ std::pair ObjectBufferPool::Ge std::lock_guard lock(pool_mutex_); if (get_buffer_state_.count(object_id) == 0) { plasma::ObjectBuffer object_buffer; - RAY_CHECK_OK(store_client_.Get(&object_id, 1, 0, &object_buffer)); + RAY_CHECK_OK( + store_client_.Get(&object_id, 1, 0, &object_buffer, /*is_from_worker=*/false)); if (object_buffer.data == nullptr) { RAY_LOG(INFO) << "Failed to get a chunk of the object: " << object_id diff --git a/src/ray/object_manager/object_manager.cc b/src/ray/object_manager/object_manager.cc index 448245e012ee..fc768fb20a31 100644 --- a/src/ray/object_manager/object_manager.cc +++ b/src/ray/object_manager/object_manager.cc @@ -834,6 +834,9 @@ void ObjectManager::FillObjectStoreStats(rpc::GetNodeStatsReply *reply) const { stats->set_object_store_bytes_used(used_memory_); stats->set_object_store_bytes_avail(config_.object_store_memory); stats->set_num_local_objects(local_objects_.size()); + if (plasma::plasma_store_runner) { + stats->set_referenced_bytes(plasma::plasma_store_runner->GetReferencedBytes()); + } } void ObjectManager::Tick(const boost::system::error_code &e) { diff --git a/src/ray/object_manager/plasma/client.cc b/src/ray/object_manager/plasma/client.cc index a5429d985f91..9b9bb5408df4 100644 --- a/src/ray/object_manager/plasma/client.cc +++ b/src/ray/object_manager/plasma/client.cc @@ -121,10 +121,10 @@ class PlasmaClient::Impl : public std::enable_shared_from_this *data, int device_num); Status Get(const std::vector &object_ids, int64_t timeout_ms, - std::vector *object_buffers); + std::vector *object_buffers, bool is_from_worker); Status Get(const ObjectID *object_ids, int64_t num_objects, int64_t timeout_ms, - ObjectBuffer *object_buffers); + ObjectBuffer *object_buffers, bool is_from_worker); Status Release(const ObjectID &object_id); @@ -172,7 +172,7 @@ class PlasmaClient::Impl : public std::enable_shared_from_this( const ObjectID &, const std::shared_ptr &)> &wrap_buffer, - ObjectBuffer *object_buffers); + ObjectBuffer *object_buffers, bool is_from_worker); uint8_t *LookupMmappedFile(MEMFD_TYPE store_fd_val); @@ -362,7 +362,7 @@ Status PlasmaClient::Impl::GetBuffers( const ObjectID *object_ids, int64_t num_objects, int64_t timeout_ms, const std::function( const ObjectID &, const std::shared_ptr &)> &wrap_buffer, - ObjectBuffer *object_buffers) { + ObjectBuffer *object_buffers, bool is_from_worker) { // Fill out the info for the objects that are already in use locally. bool all_present = true; for (int64_t i = 0; i < num_objects; ++i) { @@ -409,7 +409,8 @@ Status PlasmaClient::Impl::GetBuffers( // If we get here, then the objects aren't all currently in use by this // client, so we need to send a request to the plasma store. - RAY_RETURN_NOT_OK(SendGetRequest(store_conn_, &object_ids[0], num_objects, timeout_ms)); + RAY_RETURN_NOT_OK(SendGetRequest(store_conn_, &object_ids[0], num_objects, timeout_ms, + is_from_worker)); std::vector buffer; RAY_RETURN_NOT_OK(PlasmaReceive(store_conn_, MessageType::PlasmaGetReply, &buffer)); std::vector received_object_ids(num_objects); @@ -470,7 +471,8 @@ Status PlasmaClient::Impl::GetBuffers( } Status PlasmaClient::Impl::Get(const std::vector &object_ids, - int64_t timeout_ms, std::vector *out) { + int64_t timeout_ms, std::vector *out, + bool is_from_worker) { std::lock_guard guard(client_mutex_); const auto wrap_buffer = [=](const ObjectID &object_id, @@ -479,16 +481,19 @@ Status PlasmaClient::Impl::Get(const std::vector &object_ids, }; const size_t num_objects = object_ids.size(); *out = std::vector(num_objects); - return GetBuffers(&object_ids[0], num_objects, timeout_ms, wrap_buffer, &(*out)[0]); + return GetBuffers(&object_ids[0], num_objects, timeout_ms, wrap_buffer, &(*out)[0], + is_from_worker); } Status PlasmaClient::Impl::Get(const ObjectID *object_ids, int64_t num_objects, - int64_t timeout_ms, ObjectBuffer *out) { + int64_t timeout_ms, ObjectBuffer *out, + bool is_from_worker) { std::lock_guard guard(client_mutex_); const auto wrap_buffer = [](const ObjectID &object_id, const std::shared_ptr &buffer) { return buffer; }; - return GetBuffers(object_ids, num_objects, timeout_ms, wrap_buffer, out); + return GetBuffers(object_ids, num_objects, timeout_ms, wrap_buffer, out, + is_from_worker); } Status PlasmaClient::Impl::MarkObjectUnused(const ObjectID &object_id) { @@ -753,13 +758,14 @@ Status PlasmaClient::TryCreateImmediately(const ObjectID &object_id, } Status PlasmaClient::Get(const std::vector &object_ids, int64_t timeout_ms, - std::vector *object_buffers) { - return impl_->Get(object_ids, timeout_ms, object_buffers); + std::vector *object_buffers, bool is_from_worker) { + return impl_->Get(object_ids, timeout_ms, object_buffers, is_from_worker); } Status PlasmaClient::Get(const ObjectID *object_ids, int64_t num_objects, - int64_t timeout_ms, ObjectBuffer *object_buffers) { - return impl_->Get(object_ids, num_objects, timeout_ms, object_buffers); + int64_t timeout_ms, ObjectBuffer *object_buffers, + bool is_from_worker) { + return impl_->Get(object_ids, num_objects, timeout_ms, object_buffers, is_from_worker); } Status PlasmaClient::Release(const ObjectID &object_id) { diff --git a/src/ray/object_manager/plasma/client.h b/src/ray/object_manager/plasma/client.h index e88a9eb138a1..703250bd23b0 100644 --- a/src/ray/object_manager/plasma/client.h +++ b/src/ray/object_manager/plasma/client.h @@ -161,9 +161,10 @@ class PlasmaClient { /// \param timeout_ms The amount of time in milliseconds to wait before this /// request times out. If this value is -1, then no timeout is set. /// \param[out] object_buffers The object results. + /// \param is_from_worker Whether or not if the Get request comes from a Ray workers. /// \return The return status. Status Get(const std::vector &object_ids, int64_t timeout_ms, - std::vector *object_buffers); + std::vector *object_buffers, bool is_from_worker); /// Deprecated variant of Get() that doesn't automatically release buffers /// when they get out of scope. @@ -173,12 +174,13 @@ class PlasmaClient { /// \param timeout_ms The amount of time in milliseconds to wait before this /// request times out. If this value is -1, then no timeout is set. /// \param object_buffers An array where the results will be stored. + /// \param is_from_worker Whether or not if the Get request comes from a Ray workers. /// \return The return status. /// /// The caller is responsible for releasing any retrieved objects, but it /// should not release objects that were not retrieved. Status Get(const ObjectID *object_ids, int64_t num_objects, int64_t timeout_ms, - ObjectBuffer *object_buffers); + ObjectBuffer *object_buffers, bool is_from_worker); /// Tell Plasma that the client no longer needs the object. This should be /// called after Get() or Create() when the client is done with the object. diff --git a/src/ray/object_manager/plasma/plasma.fbs b/src/ray/object_manager/plasma/plasma.fbs index 3816de79e842..d8fda975b998 100644 --- a/src/ray/object_manager/plasma/plasma.fbs +++ b/src/ray/object_manager/plasma/plasma.fbs @@ -210,6 +210,8 @@ table PlasmaGetRequest { object_ids: [string]; // The number of milliseconds before the request should timeout. timeout_ms: long; + // Whether or not the get request is from the core worker. It is used to record how many objects are referenced by core workers. + is_from_worker: bool; } table PlasmaGetReply { diff --git a/src/ray/object_manager/plasma/protocol.cc b/src/ray/object_manager/plasma/protocol.cc index 8c3164d6a7df..c3b5b55ee1d5 100644 --- a/src/ray/object_manager/plasma/protocol.cc +++ b/src/ray/object_manager/plasma/protocol.cc @@ -553,16 +553,16 @@ Status ReadEvictReply(uint8_t *data, size_t size, int64_t &num_bytes) { // Get messages. Status SendGetRequest(const std::shared_ptr &store_conn, - const ObjectID *object_ids, int64_t num_objects, - int64_t timeout_ms) { + const ObjectID *object_ids, int64_t num_objects, int64_t timeout_ms, + bool is_from_worker) { flatbuffers::FlatBufferBuilder fbb; auto message = fb::CreatePlasmaGetRequest( - fbb, ToFlatbuffer(&fbb, object_ids, num_objects), timeout_ms); + fbb, ToFlatbuffer(&fbb, object_ids, num_objects), timeout_ms, is_from_worker); return PlasmaSend(store_conn, MessageType::PlasmaGetRequest, &fbb, message); } Status ReadGetRequest(uint8_t *data, size_t size, std::vector &object_ids, - int64_t *timeout_ms) { + int64_t *timeout_ms, bool *is_from_worker) { RAY_DCHECK(data); auto message = flatbuffers::GetRoot(data); RAY_DCHECK(VerifyFlatbuffer(message, data, size)); @@ -571,6 +571,7 @@ Status ReadGetRequest(uint8_t *data, size_t size, std::vector &object_ object_ids.push_back(ObjectID::FromBinary(object_id)); } *timeout_ms = message->timeout_ms(); + *is_from_worker = message->is_from_worker(); return Status::OK(); } diff --git a/src/ray/object_manager/plasma/protocol.h b/src/ray/object_manager/plasma/protocol.h index a8ba71b4621f..f5baf03ec955 100644 --- a/src/ray/object_manager/plasma/protocol.h +++ b/src/ray/object_manager/plasma/protocol.h @@ -128,11 +128,11 @@ Status ReadSealReply(uint8_t *data, size_t size, ObjectID *object_id); /* Plasma Get message functions. */ Status SendGetRequest(const std::shared_ptr &store_conn, - const ObjectID *object_ids, int64_t num_objects, - int64_t timeout_ms); + const ObjectID *object_ids, int64_t num_objects, int64_t timeout_ms, + bool is_from_worker); Status ReadGetRequest(uint8_t *data, size_t size, std::vector &object_ids, - int64_t *timeout_ms); + int64_t *timeout_ms, bool *is_from_worker); Status SendGetReply(const std::shared_ptr &client, ObjectID object_ids[], std::unordered_map &plasma_objects, diff --git a/src/ray/object_manager/plasma/store.cc b/src/ray/object_manager/plasma/store.cc index e101c5a9b71a..9627c9dc3682 100644 --- a/src/ray/object_manager/plasma/store.cc +++ b/src/ray/object_manager/plasma/store.cc @@ -69,7 +69,7 @@ namespace plasma { struct GetRequest { GetRequest(boost::asio::io_service &io_context, const std::shared_ptr &client, - const std::vector &object_ids); + const std::vector &object_ids, bool is_from_worker); /// The client that called get. std::shared_ptr client; /// The object IDs involved in this request. This is used in the reply. @@ -82,6 +82,9 @@ struct GetRequest { /// The number of object requests in this wait request that are already /// satisfied. int64_t num_satisfied; + /// Whether or not the request comes from the core worker. It is used to track the size + /// of total objects that are referenced by core worker. + bool is_from_worker; void AsyncWait(int64_t timeout_ms, std::function on_timeout) { @@ -100,11 +103,12 @@ struct GetRequest { GetRequest::GetRequest(boost::asio::io_service &io_context, const std::shared_ptr &client, - const std::vector &object_ids) + const std::vector &object_ids, bool is_from_worker) : client(client), object_ids(object_ids.begin(), object_ids.end()), objects(object_ids.size()), num_satisfied(0), + is_from_worker(is_from_worker), timer_(io_context) { std::unordered_set unique_ids(object_ids.begin(), object_ids.end()); num_objects_to_wait_for = unique_ids.size(); @@ -393,6 +397,9 @@ void PlasmaStore::ReturnFromGet(GetRequest *get_req) { fds_to_send.insert(fd); store_fds.push_back(fd); mmap_sizes.push_back(GetMmapSize(fd)); + if (get_req->is_from_worker) { + total_referenced_bytes_ += object.data_size + object.metadata_size; + } } } // Send the get reply to the client. @@ -465,9 +472,9 @@ void PlasmaStore::UpdateObjectGetRequests(const ObjectID &object_id) { void PlasmaStore::ProcessGetRequest(const std::shared_ptr &client, const std::vector &object_ids, - int64_t timeout_ms) { + int64_t timeout_ms, bool is_from_worker) { // Create a get request for this object. - auto get_req = new GetRequest(io_context_, client, object_ids); + auto get_req = new GetRequest(io_context_, client, object_ids, is_from_worker); for (auto object_id : object_ids) { // Check if this object is already present // locally. If so, record that the object is being used and mark it as accounted for. @@ -894,8 +901,10 @@ Status PlasmaStore::ProcessMessage(const std::shared_ptr &client, case fb::MessageType::PlasmaGetRequest: { std::vector object_ids_to_get; int64_t timeout_ms; - RAY_RETURN_NOT_OK(ReadGetRequest(input, input_size, object_ids_to_get, &timeout_ms)); - ProcessGetRequest(client, object_ids_to_get, timeout_ms); + bool is_from_worker; + RAY_RETURN_NOT_OK(ReadGetRequest(input, input_size, object_ids_to_get, &timeout_ms, + &is_from_worker)); + ProcessGetRequest(client, object_ids_to_get, timeout_ms, is_from_worker); } break; case fb::MessageType::PlasmaReleaseRequest: { RAY_RETURN_NOT_OK(ReadReleaseRequest(input, input_size, &object_id)); @@ -1020,6 +1029,11 @@ void PlasmaStore::ReplyToCreateClient(const std::shared_ptr &client, } } +int64_t PlasmaStore::GetReferencedBytes() { + std::lock_guard guard(mutex_); + return total_referenced_bytes_; +} + bool PlasmaStore::IsObjectSpillable(const ObjectID &object_id) { // The lock is acquired when a request is received to the plasma store. // recursive mutex is used here to allow diff --git a/src/ray/object_manager/plasma/store.h b/src/ray/object_manager/plasma/store.h index 214cf9763bf6..cbe4e221acdb 100644 --- a/src/ray/object_manager/plasma/store.h +++ b/src/ray/object_manager/plasma/store.h @@ -139,7 +139,8 @@ class PlasmaStore { /// \param object_ids Object IDs of the objects to be gotten. /// \param timeout_ms The timeout for the get request in milliseconds. void ProcessGetRequest(const std::shared_ptr &client, - const std::vector &object_ids, int64_t timeout_ms); + const std::vector &object_ids, int64_t timeout_ms, + bool is_from_worker); /// Seal a vector of objects. The objects are now immutable and can be accessed with /// get. @@ -190,6 +191,9 @@ class PlasmaStore { /// before the object is pinned by raylet for the first time. bool IsObjectSpillable(const ObjectID &object_id); + /// Return the plasma object bytes that are referenced by core workers. + int64_t GetReferencedBytes(); + void SetNotificationListener( const std::shared_ptr ¬ification_listener) { notification_listener_ = notification_listener; @@ -316,6 +320,9 @@ class PlasmaStore { std::recursive_mutex mutex_; size_t num_bytes_in_use_ = 0; + + /// Total plasma object bytes that are referenced by core workers. + int64_t total_referenced_bytes_ = 0; }; } // namespace plasma diff --git a/src/ray/object_manager/plasma/store_runner.cc b/src/ray/object_manager/plasma/store_runner.cc index 34e08080cced..6923e449bb24 100644 --- a/src/ray/object_manager/plasma/store_runner.cc +++ b/src/ray/object_manager/plasma/store_runner.cc @@ -123,6 +123,8 @@ bool PlasmaStoreRunner::IsPlasmaObjectSpillable(const ObjectID &object_id) { return store_->IsObjectSpillable(object_id); } +int64_t PlasmaStoreRunner::GetReferencedBytes() { return store_->GetReferencedBytes(); } + std::unique_ptr plasma_store_runner; } // namespace plasma diff --git a/src/ray/object_manager/plasma/store_runner.h b/src/ray/object_manager/plasma/store_runner.h index 7ac7be59bbc5..61a00fc683ad 100644 --- a/src/ray/object_manager/plasma/store_runner.h +++ b/src/ray/object_manager/plasma/store_runner.h @@ -22,6 +22,8 @@ class PlasmaStoreRunner { } bool IsPlasmaObjectSpillable(const ObjectID &object_id); + int64_t GetReferencedBytes(); + void GetAvailableMemoryAsync(std::function callback) const { main_service_.post([this, callback]() { store_->GetAvailableMemory(callback); }); } diff --git a/src/ray/protobuf/common.proto b/src/ray/protobuf/common.proto index de3c6d6a2138..844f44bea723 100644 --- a/src/ray/protobuf/common.proto +++ b/src/ray/protobuf/common.proto @@ -428,8 +428,6 @@ message CoreWorkerStats { uint32 pid = 22; // The worker type. WorkerType worker_type = 23; - // Total plasma object bytes that are referenced by this core worker. - int64 referenced_bytes = 24; } message MetricPoint { diff --git a/src/ray/raylet/node_manager.cc b/src/ray/raylet/node_manager.cc index 01c485ecb02d..5084cc50be14 100644 --- a/src/ray/raylet/node_manager.cc +++ b/src/ray/raylet/node_manager.cc @@ -2383,7 +2383,9 @@ bool NodeManager::GetObjectsFromPlasma(const std::vector &object_ids, // heavy load, then this request can still block the NodeManager event loop // since we must wait for the plasma store's reply. We should consider using // an `AsyncGet` instead. - if (!store_client_.Get(object_ids, /*timeout_ms=*/0, &plasma_results).ok()) { + if (!store_client_ + .Get(object_ids, /*timeout_ms=*/0, &plasma_results, /*is_from_worker=*/false) + .ok()) { return false; } @@ -2509,14 +2511,11 @@ void NodeManager::HandleGetNodeStats(const rpc::GetNodeStatsRequest &node_stats_ request.set_intended_worker_id(worker->WorkerId().Binary()); request.set_include_memory_info(node_stats_request.include_memory_info()); worker->rpc_client()->GetCoreWorkerStats( - request, [this, reply, worker, all_workers, driver_ids, send_reply_callback]( + request, [reply, worker, all_workers, driver_ids, send_reply_callback]( const ray::Status &status, const rpc::GetCoreWorkerStatsReply &r) { - total_referenced_bytes_ += r.core_worker_stats().referenced_bytes(); reply->add_core_workers_stats()->MergeFrom(r.core_worker_stats()); reply->set_num_workers(reply->num_workers() + 1); if (reply->num_workers() == all_workers.size()) { - auto stats = reply->mutable_store_stats(); - stats->set_referenced_bytes(total_referenced_bytes_); send_reply_callback(Status::OK(), nullptr, nullptr); } }); diff --git a/src/ray/raylet/node_manager.h b/src/ray/raylet/node_manager.h index cbb4af6ce677..606dc3ac6fa7 100644 --- a/src/ray/raylet/node_manager.h +++ b/src/ray/raylet/node_manager.h @@ -935,10 +935,6 @@ class NodeManager : public rpc::NodeManagerServiceHandler, /// Managers all bundle-related operations. std::shared_ptr placement_group_resource_manager_; - - /// The total bytes size of plasma objects that are referenced by core workers. - /// It is accumulative. - int64_t total_referenced_bytes_ = 0; }; } // namespace raylet From ea8d0cb11a17bdb9643de204e068dbfe3fde0160 Mon Sep 17 00:00:00 2001 From: SangBin Cho Date: Tue, 2 Feb 2021 12:09:47 -0800 Subject: [PATCH 3/5] d --- .../workloads/output.csv | 3 + .../workloads/test_dask.py | 189 +++++++++++++++++ release/raydp_tests/dask_on_ray/test_dask.py | 193 ++++++++++++++++++ 3 files changed, 385 insertions(+) create mode 100644 release/data_processing_tests/workloads/output.csv create mode 100644 release/data_processing_tests/workloads/test_dask.py create mode 100644 release/raydp_tests/dask_on_ray/test_dask.py diff --git a/release/data_processing_tests/workloads/output.csv b/release/data_processing_tests/workloads/output.csv new file mode 100644 index 000000000000..2fa7483a1b1d --- /dev/null +++ b/release/data_processing_tests/workloads/output.csv @@ -0,0 +1,3 @@ +system,operation,num_nodes,nbytes,npartitions,duration +ray,sort,1,1000000000,50,38.00257587432861 +ray,sort,1,1000000000,100,65.93781399726868 diff --git a/release/data_processing_tests/workloads/test_dask.py b/release/data_processing_tests/workloads/test_dask.py new file mode 100644 index 000000000000..78a13dd0ec8d --- /dev/null +++ b/release/data_processing_tests/workloads/test_dask.py @@ -0,0 +1,189 @@ +import glob +import ray +import dask +import dask.dataframe as dd +import json +import pandas as pd +import numpy as np +from ray.util.dask import ray_dask_get_sync +import os.path +import csv +import fastparquet + +from dask.distributed import Client +from dask.distributed import wait +import cProfile, pstats, io +from pstats import SortKey + +import time + +DATA_DIR = "~/dask-on-ray-data" +# DATA_DIR = "/obj-data" + + +def load_dataset(nbytes, npartitions, sort): + num_bytes_per_partition = nbytes // npartitions + filenames = [] + + @ray.remote + def foo(i): + filename = "df-{}-{}-{}.parquet.gzip".format( + "sort" if sort else "groupby", num_bytes_per_partition, i) + filename = os.path.join(DATA_DIR, filename) + print("Partition file", filename) + if not os.path.exists(filename): + if sort: + nrows = num_bytes_per_partition // 8 + print("Allocating dataset with {} rows".format(nrows)) + dataset = pd.DataFrame( + np.random.randint( + 0, + np.iinfo(np.int64).max, + size=(nrows, 1), + dtype=np.int64), + columns=['a']) + else: + nrows = num_bytes_per_partition // (8 * 2) + print("Allocating dataset with {} rows".format(nrows)) + dataset = pd.DataFrame( + np.random.randint(0, 100, size=(nrows, 2), dtype=np.int64), + columns=['a', 'b']) + print("Done allocating") + dataset.to_parquet(filename, compression='gzip') + print("Done writing to disk") + return filename + + for i in range(npartitions): + filenames.append(foo.remote(i)) + filenames = ray.get(filenames) + + df = dd.read_parquet(filenames) + return df + + +def trial(nbytes, n_partitions, sort, generate_only): + df = load_dataset(nbytes, n_partitions, sort) + pr = cProfile.Profile() + if generate_only: + return + + times = [] + start = time.time() + for i in range(1): + print("Trial {} start".format(i)) + trial_start = time.time() + + if sort: + # pr.enable() + a = df.set_index('a', shuffle='tasks', max_branch=10**9) + # a.visualize(filename=f'a-{i}.svg') + a.head(10, npartitions=-1) + # pr.disable() + else: + df.groupby('b').a.mean().compute() + + trial_end = time.time() + duration = trial_end - trial_start + times.append(duration) + print("Trial {} done after {}".format(i, duration)) + # sortby = SortKey.CUMULATIVE + # ps = pstats.Stats(pr).sort_stats(sortby) + # ps.dump_stats("ray_profile_data") + + if time.time() - start > 60 and i > 0: + break + return times + + +if __name__ == '__main__': + import argparse + parser = argparse.ArgumentParser() + + parser.add_argument("--nbytes", type=int, default=1_000_000) + parser.add_argument("--npartitions", type=int, default=100, required=False) + # Max partition size is 1GB. + parser.add_argument( + "--max-partition-size", type=int, default=1000_000_000, required=False) + parser.add_argument("--num-nodes", type=int, default=1) + parser.add_argument("--sort", action="store_true") + parser.add_argument("--timeline", action="store_true") + parser.add_argument("--dask", action="store_true") + parser.add_argument("--ray", action="store_true") + parser.add_argument("--dask-tasks", action="store_true") + parser.add_argument("--generate-only", action="store_true") + parser.add_argument("--clear-old-data", action="store_true") + args = parser.parse_args() + + if args.clear_old_data: + print(f"Clearing old data from {DATA_DIR}.") + files = glob.glob(os.path.join(DATA_DIR, "*.parquet.gzip")) + for f in files: + os.remove(f) + + if args.ray: + args.dask_tasks = True + + if args.dask_tasks: + print("Using task-based Dask shuffle") + dask.config.set(shuffle='tasks') + else: + print("Using disk-based Dask shuffle") + + if args.dask: + client = Client('127.0.0.1:8786') + ray.init(address='auto') + if args.ray: + ray.init(address="auto") + # ray.init( + # num_cpus=16, + # _system_config={ + # "max_io_workers": 1, + # "object_spilling_config": json.dumps( + # { + # "type": "filesystem", + # "params": { + # "directory_path": "/tmp/spill" + # } + # }, + # separators=(",", ":")) + # }) + dask.config.set(scheduler=ray_dask_get_sync) + + system = "dask" if args.dask else "ray" + + # print(system, trial(1000, 10, args.sort, args.generate_only)) + print("WARMUP DONE") + + npartitions = args.npartitions + if args.nbytes // npartitions > args.max_partition_size: + npartitions = args.nbytes // args.max_partition_size + + output = trial(args.nbytes, npartitions, args.sort, args.generate_only) + print("{} mean over {} trials: {} +- {}".format(system, len(output), + np.mean(output), + np.std(output))) + + write_header = not os.path.exists("output.csv") or os.path.getsize( + "output.csv") == 0 + with open("output.csv", "a+") as csvfile: + fieldnames = [ + "system", "operation", "num_nodes", "nbytes", "npartitions", + "duration" + ] + writer = csv.DictWriter(csvfile, fieldnames=fieldnames) + if write_header: + writer.writeheader() + row = { + "operation": "sort" if args.sort else "groupby", + "num_nodes": args.num_nodes, + "nbytes": args.nbytes, + "npartitions": npartitions, + } + for output in output: + row["system"] = system + row["duration"] = output + writer.writerow(row) + + if args.timeline: + time.sleep(1) + ray.timeline(filename="dask.json") \ No newline at end of file diff --git a/release/raydp_tests/dask_on_ray/test_dask.py b/release/raydp_tests/dask_on_ray/test_dask.py new file mode 100644 index 000000000000..cfec8daa3c9d --- /dev/null +++ b/release/raydp_tests/dask_on_ray/test_dask.py @@ -0,0 +1,193 @@ +import glob +import ray +import dask +import dask.dataframe as dd +import json +import pandas as pd +import numpy as np +from ray.util.dask import ray_dask_get_sync +import os.path +import csv +import fastparquet + +from dask.distributed import Client +from dask.distributed import wait +import cProfile, pstats, io +from pstats import SortKey + +import time + +# DATA_DIR = "~/dask-on-ray-data" +DATA_DIR = "/obj-data" + + +def load_dataset(nbytes, npartitions, sort)::/ + num_bytes_per_partition = nbytes // npartitions + filenames = [] + + @ray.remote + def foo(i): + filename = "df-{}-{}-{}.parquet.gzip".format( + "sort" if sort else "groupby", num_bytes_per_partition, i) + filename = os.path.join(DATA_DIR, filename) + print("Partition file", filename) + if not os.path.exists(filename): + if sort: + nrows = num_bytes_per_partition // 8 + print("Allocating dataset with {} rows".format(nrows)) + dataset = pd.DataFrame( + np.random.randint( + 0, + np.iinfo(np.int64).max, + size=(nrows, 1), + dtype=np.int64), + columns=['a']) + else: + nrows = num_bytes_per_partition // (8 * 2) + print("Allocating dataset with {} rows".format(nrows)) + dataset = pd.DataFrame( + np.random.randint(0, 100, size=(nrows, 2), dtype=np.int64), + columns=['a', 'b']) + print("Done allocating") + dataset.to_parquet(filename, compression='gzip') + print("Done writing to disk") + return filename + + for i in range(npartitions): + filenames.append(foo.remote(i)) + filenames = ray.get(filenames) + + df = dd.read_parquet(filenames) + import time + print("parquet read") + time.sleep(20) + print("sleep done.") + return df + + +def trial(nbytes, n_partitions, sort, generate_only): + df = load_dataset(nbytes, n_partitions, sort) + pr = cProfile.Profile() + if generate_only: + return + + times = [] + start = time.time() + for i in range(1): + print("Trial {} start".format(i)) + trial_start = time.time() + + if sort: + # pr.enable() + a = df.set_index('a', shuffle='tasks', max_branch=10**9) + # a.visualize(filename=f'a-{i}.svg') + a.head(10, npartitions=-1) + # pr.disable() + else: + df.groupby('b').a.mean().compute() + + trial_end = time.time() + duration = trial_end - trial_start + times.append(duration) + print("Trial {} done after {}".format(i, duration)) + # sortby = SortKey.CUMULATIVE + # ps = pstats.Stats(pr).sort_stats(sortby) + # ps.dump_stats("ray_profile_data") + + if time.time() - start > 60 and i > 0: + break + return times + + +if __name__ == '__main__': + import argparse + parser = argparse.ArgumentParser() + + parser.add_argument("--nbytes", type=int, default=1_000_000) + parser.add_argument("--npartitions", type=int, default=100, required=False) + # Max partition size is 1GB. + parser.add_argument( + "--max-partition-size", type=int, default=1000_000_000, required=False) + parser.add_argument("--num-nodes", type=int, default=1) + parser.add_argument("--sort", action="store_true") + parser.add_argument("--timeline", action="store_true") + parser.add_argument("--dask", action="store_true") + parser.add_argument("--ray", action="store_true") + parser.add_argument("--dask-tasks", action="store_true") + parser.add_argument("--generate-only", action="store_true") + parser.add_argument("--clear-old-data", action="store_true") + args = parser.parse_args() + + if args.clear_old_data: + print(f"Clearing old data from {DATA_DIR}.") + files = glob.glob(os.path.join(DATA_DIR, "*.parquet.gzip")) + for f in files: + os.remove(f) + + if args.ray: + args.dask_tasks = True + + if args.dask_tasks: + print("Using task-based Dask shuffle") + dask.config.set(shuffle='tasks') + else: + print("Using disk-based Dask shuffle") + + if args.dask: + client = Client('127.0.0.1:8786') + ray.init(address='auto') + if args.ray: + ray.init(address="auto") + # ray.init( + # num_cpus=16, + # _system_config={ + # "max_io_workers": 1, + # "object_spilling_config": json.dumps( + # { + # "type": "filesystem", + # "params": { + # "directory_path": "/tmp/spill" + # } + # }, + # separators=(",", ":")) + # }) + dask.config.set(scheduler=ray_dask_get_sync) + + system = "dask" if args.dask else "ray" + + # print(system, trial(1000, 10, args.sort, args.generate_only)) + print("WARMUP DONE") + + npartitions = args.npartitions + if args.nbytes // npartitions > args.max_partition_size: + npartitions = args.nbytes // args.max_partition_size + + output = trial(args.nbytes, npartitions, args.sort, args.generate_only) + print("{} mean over {} trials: {} +- {}".format(system, len(output), + np.mean(output), + np.std(output))) + + write_header = not os.path.exists("output.csv") or os.path.getsize( + "output.csv") == 0 + with open("output.csv", "a+") as csvfile: + fieldnames = [ + "system", "operation", "num_nodes", "nbytes", "npartitions", + "duration" + ] + writer = csv.DictWriter(csvfile, fieldnames=fieldnames) + if write_header: + writer.writeheader() + row = { + "operation": "sort" if args.sort else "groupby", + "num_nodes": args.num_nodes, + "nbytes": args.nbytes, + "npartitions": npartitions, + } + for output in output: + row["system"] = system + row["duration"] = output + writer.writerow(row) + + if args.timeline: + time.sleep(1) + ray.timeline(filename="dask.json") \ No newline at end of file From 09ad4ae25ba877dc0f822c51396ad394a343a3c7 Mon Sep 17 00:00:00 2001 From: SangBin Cho Date: Tue, 2 Feb 2021 15:41:39 -0800 Subject: [PATCH 4/5] done --- release/data_processing_tests/workloads/test_dask.py | 3 ++- release/raydp_tests/dask_on_ray/test_dask.py | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/release/data_processing_tests/workloads/test_dask.py b/release/data_processing_tests/workloads/test_dask.py index 78a13dd0ec8d..a3b924ddcefb 100644 --- a/release/data_processing_tests/workloads/test_dask.py +++ b/release/data_processing_tests/workloads/test_dask.py @@ -18,6 +18,7 @@ import time DATA_DIR = "~/dask-on-ray-data" + # DATA_DIR = "/obj-data" @@ -186,4 +187,4 @@ def trial(nbytes, n_partitions, sort, generate_only): if args.timeline: time.sleep(1) - ray.timeline(filename="dask.json") \ No newline at end of file + ray.timeline(filename="dask.json") diff --git a/release/raydp_tests/dask_on_ray/test_dask.py b/release/raydp_tests/dask_on_ray/test_dask.py index cfec8daa3c9d..b17415b4846a 100644 --- a/release/raydp_tests/dask_on_ray/test_dask.py +++ b/release/raydp_tests/dask_on_ray/test_dask.py @@ -21,7 +21,7 @@ DATA_DIR = "/obj-data" -def load_dataset(nbytes, npartitions, sort)::/ +def load_dataset(nbytes, npartitions, sort): num_bytes_per_partition = nbytes // npartitions filenames = [] From 5af5ef54fd318e7954e0d8915a431722927c9c1d Mon Sep 17 00:00:00 2001 From: SangBin Cho Date: Tue, 2 Feb 2021 23:41:58 -0800 Subject: [PATCH 5/5] d. --- python/ray/util/dask/scheduler.py | 2 + release/raydp_tests/dask_on_ray/a-0.svg | 82644 +++++++++++++++++ release/raydp_tests/dask_on_ray/a.txt | 6 + release/raydp_tests/dask_on_ray/analyze.py | 60 + release/raydp_tests/dask_on_ray/output.csv | 45 + release/raydp_tests/dask_on_ray/result.txt | 1248 + release/raydp_tests/dask_on_ray/test_dask.py | 93 +- src/ray/object_manager/plasma/store.cc | 1 + 8 files changed, 84082 insertions(+), 17 deletions(-) create mode 100644 release/raydp_tests/dask_on_ray/a-0.svg create mode 100644 release/raydp_tests/dask_on_ray/a.txt create mode 100644 release/raydp_tests/dask_on_ray/analyze.py create mode 100644 release/raydp_tests/dask_on_ray/output.csv create mode 100644 release/raydp_tests/dask_on_ray/result.txt diff --git a/python/ray/util/dask/scheduler.py b/python/ray/util/dask/scheduler.py index 0614d35641ec..8737fc491305 100644 --- a/python/ray/util/dask/scheduler.py +++ b/python/ray/util/dask/scheduler.py @@ -275,6 +275,8 @@ def _rayify_task( # Ray properly tracks the object dependencies between Ray tasks. object_refs, repack = unpack_object_refs(args, deps) # Submit the task using a wrapper function. + for ref in object_refs: + print(f"[GET] object_id:{ref.hex()} size:0 func:{key}") object_ref = dask_task_wrapper.options(name=f"dask:{key!s}").remote( func, repack, key, ray_pretask_cbs, ray_posttask_cbs, *object_refs) diff --git a/release/raydp_tests/dask_on_ray/a-0.svg b/release/raydp_tests/dask_on_ray/a-0.svg new file mode 100644 index 000000000000..962e1773f794 --- /dev/null +++ b/release/raydp_tests/dask_on_ray/a-0.svg @@ -0,0 +1,82644 @@ + + + + + + + + + +5326898693109738271 + +assign + + + +-5552841363955063783 + +0 + + + +5326898693109738271->-5552841363955063783 + + + + + +3561848877138230675 + +simple-shuffle-group + + + +-5552841363955063783->3561848877138230675 + + + + + +8118300845140450747 + +0 + + + +8118300845140450747->5326898693109738271 + + + + + +7739510092502631059 + +getitem + + + +8118300845140450747->7739510092502631059 + + + + + +3913254328728358566 + +0 + + + +3913254328728358566->5326898693109738271 + + + + + +4305073209093423396 + +assign + + + +-5552841363953981258 + +1 + + + +4305073209093423396->-5552841363953981258 + + + + + +391848673754780200 + +simple-shuffle-group + + + +-5552841363953981258->391848673754780200 + + + + + +8118300845141533272 + +1 + + + +8118300845141533272->4305073209093423396 + + + + + +6638842773940018984 + +getitem + + + +8118300845141533272->6638842773940018984 + + + + + +3913254328727276041 + +1 + + + +3913254328727276041->4305073209093423396 + + + + + +-8732128262403687595 + +assign + + + +-5552841363957228833 + +2 + + + +-8732128262403687595->-5552841363957228833 + + + + + +4648106040364701225 + +simple-shuffle-group + + + +-5552841363957228833->4648106040364701225 + + + + + +3913254328726193516 + +2 + + + +3913254328726193516->-8732128262403687595 + + + + + +8118300845142615797 + +2 + + + +8118300845142615797->-8732128262403687595 + + + + + +1404753993089809193 + +getitem + + + +8118300845142615797->1404753993089809193 + + + + + +6489435645788862346 + +assign + + + +-5552841363956146308 + +3 + + + +6489435645788862346->-5552841363956146308 + + + + + +4658027425283123550 + +simple-shuffle-group + + + +-5552841363956146308->4658027425283123550 + + + + + +8118300845143698322 + +3 + + + +8118300845143698322->6489435645788862346 + + + + + +8836797070869143134 + +getitem + + + +8118300845143698322->8836797070869143134 + + + + + +3913254328725110991 + +3 + + + +3913254328725110991->6489435645788862346 + + + + + +-6459787997608901445 + +assign + + + +-5552841363959393883 + +4 + + + +-6459787997608901445->-5552841363959393883 + + + + + +-3373206642788804425 + +simple-shuffle-group + + + +-5552841363959393883->-3373206642788804425 + + + + + +3913254328724028466 + +4 + + + +3913254328724028466->-6459787997608901445 + + + + + +8118300845144780847 + +4 + + + +8118300845144780847->-6459787997608901445 + + + + + +2382585336333778359 + +getitem + + + +8118300845144780847->2382585336333778359 + + + + + +-7635948587999784720 + +assign + + + +-5552841363958311358 + +5 + + + +-7635948587999784720->-5552841363958311358 + + + + + +-4479352522949166500 + +simple-shuffle-group + + + +-5552841363958311358->-4479352522949166500 + + + + + +8118300845145863372 + +5 + + + +8118300845145863372->-7635948587999784720 + + + + + +-7642776713948807332 + +getitem + + + +8118300845145863372->-7642776713948807332 + + + + + +3913254328722945941 + +5 + + + +3913254328722945941->-7635948587999784720 + + + + + +-3887674053199092495 + +assign + + + +-5552841363961558933 + +6 + + + +-3887674053199092495->-5552841363961558933 + + + + + +-1851365200229048275 + +simple-shuffle-group + + + +-5552841363961558933->-1851365200229048275 + + + + + +3913254328721863416 + +6 + + + +3913254328721863416->-3887674053199092495 + + + + + +8118300845146945897 + +6 + + + +8118300845146945897->-3887674053199092495 + + + + + +-3465338802196343507 + +getitem + + + +8118300845146945897->-3465338802196343507 + + + + + +3570004133279731846 + +assign + + + +-5552841363960476408 + +7 + + + +3570004133279731846->-5552841363960476408 + + + + + +6569962593908858050 + +simple-shuffle-group + + + +-5552841363960476408->6569962593908858050 + + + + + +3913254328720780891 + +7 + + + +3913254328720780891->3570004133279731846 + + + + + +8118300845148028422 + +7 + + + +8118300845148028422->3570004133279731846 + + + + + +3405236447878263234 + +getitem + + + +8118300845148028422->3405236447878263234 + + + + + +-4742478810160460729 + +assign + + + +-5552841363946403583 + +8 + + + +-4742478810160460729->-5552841363946403583 + + + + + +-5766413972891547541 + +simple-shuffle-group + + + +-5552841363946403583->-5766413972891547541 + + + + + +8118300845149110947 + +8 + + + +8118300845149110947->-4742478810160460729 + + + + + +682995203040558443 + +getitem + + + +8118300845149110947->682995203040558443 + + + + + +3913254328737018766 + +8 + + + +3913254328737018766->-4742478810160460729 + + + + + +-5917514269601675604 + +assign + + + +-5552841363945321058 + +9 + + + +-5917514269601675604->-5552841363945321058 + + + + + +-7871270668270266016 + +simple-shuffle-group + + + +-5552841363945321058->-7871270668270266016 + + + + + +8118300845150193472 + +9 + + + +8118300845150193472->-5917514269601675604 + + + + + +-492026381288022432 + +getitem + + + +8118300845150193472->-492026381288022432 + + + + + +3913254328735936241 + +9 + + + +3913254328735936241->-5917514269601675604 + + + + + +-1640051760618312579 + +assign + + + +-5552841363948568633 + +10 + + + +-1640051760618312579->-5552841363948568633 + + + + + +-3559868500111997791 + +simple-shuffle-group + + + +-5552841363948568633->-3559868500111997791 + + + + + +3913254328734853716 + +10 + + + +3913254328734853716->-1640051760618312579 + + + + + +8118300845151275997 + +10 + + + +8118300845151275997->-1640051760618312579 + + + + + +3238498460401228193 + +getitem + + + +8118300845151275997->3238498460401228193 + + + + + +-3646545549640598254 + +assign + + + +-5552841363947486108 + +11 + + + +-3646545549640598254->-5552841363947486108 + + + + + +-4660231804639902666 + +simple-shuffle-group + + + +-5552841363947486108->-4660231804639902666 + + + + + +8118300845152358522 + +11 + + + +8118300845152358522->-3646545549640598254 + + + + + +1695960576957278518 + +getitem + + + +8118300845152358522->1695960576957278518 + + + + + +3913254328733771191 + +11 + + + +3913254328733771191->-3646545549640598254 + + + + + +482403654283737571 + +assign + + + +-5552841363950733683 + +12 + + + +482403654283737571->-5552841363950733683 + + + + + +8401877563929125775 + +simple-shuffle-group + + + +-5552841363950733683->8401877563929125775 + + + + + +3913254328732688666 + +12 + + + +3913254328732688666->482403654283737571 + + + + + +8118300845153441047 + +12 + + + +8118300845153441047->482403654283737571 + + + + + +-5993672831596595057 + +getitem + + + +8118300845153441047->-5993672831596595057 + + + + + +-613792835058932104 + +assign + + + +-5552841363949651158 + +13 + + + +-613792835058932104->-5552841363949651158 + + + + + +5748732521581009300 + +simple-shuffle-group + + + +-5552841363949651158->5748732521581009300 + + + + + +8118300845154523572 + +13 + + + +8118300845154523572->-613792835058932104 + + + + + +-7090997268567021932 + +getitem + + + +8118300845154523572->-7090997268567021932 + + + + + +3913254328731606141 + +13 + + + +3913254328731606141->-613792835058932104 + + + + + +3204443312571361721 + +assign + + + +-5552841363952898733 + +14 + + + +3204443312571361721->-5552841363952898733 + + + + + +-7781630427718845691 + +simple-shuffle-group + + + +-5552841363952898733->-7781630427718845691 + + + + + +3913254328730523616 + +14 + + + +3913254328730523616->3204443312571361721 + + + + + +8118300845155606097 + +14 + + + +8118300845155606097->3204443312571361721 + + + + + +-1589350818223808507 + +getitem + + + +8118300845155606097->-1589350818223808507 + + + + + +2848492283808400046 + +assign + + + +-5552841363951816208 + +15 + + + +2848492283808400046->-5552841363951816208 + + + + + +-8944203705052616966 + +simple-shuffle-group + + + +-5552841363951816208->-8944203705052616966 + + + + + +3913254328729441091 + +15 + + + +3913254328729441091->2848492283808400046 + + + + + +8118300845156688622 + +15 + + + +8118300845156688622->2848492283808400046 + + + + + +-3060854971146904582 + +getitem + + + +8118300845156688622->-3060854971146904582 + + + + + +6320267094337201455 + +assign + + + +-5552841363972384183 + +16 + + + +6320267094337201455->-5552841363972384183 + + + + + +608660796933665091 + +simple-shuffle-group + + + +-5552841363972384183->608660796933665091 + + + + + +8118300845157771147 + +16 + + + +8118300845157771147->6320267094337201455 + + + + + +-4391383745347844541 + +getitem + + + +8118300845157771147->-4391383745347844541 + + + + + +3913254328745678966 + +16 + + + +3913254328745678966->6320267094337201455 + + + + + +5219653942952694580 + +assign + + + +-5552841363971301658 + +17 + + + +5219653942952694580->-5552841363971301658 + + + + + +-491696868626252584 + +simple-shuffle-group + + + +-5552841363971301658->-491696868626252584 + + + + + +8118300845158853672 + +17 + + + +8118300845158853672->5219653942952694580 + + + + + +-5492850904186251816 + +getitem + + + +8118300845158853672->-5492850904186251816 + + + + + +3913254328744596441 + +17 + + + +3913254328744596441->5219653942952694580 + + + + + +-8919850247338757211 + +assign + + + +-5552841363974549233 + +18 + + + +-8919850247338757211->-5552841363974549233 + + + + + +-7408227040647628775 + +simple-shuffle-group + + + +-5552841363974549233->-7408227040647628775 + + + + + +3913254328743513916 + +18 + + + +3913254328743513916->-8919850247338757211 + + + + + +8118300845159936197 + +18 + + + +8118300845159936197->-8919850247338757211 + + + + + +7854932871154028825 + +getitem + + + +8118300845159936197->7854932871154028825 + + + + + +7343217716023499930 + +assign + + + +-5552841363973466708 + +19 + + + +7343217716023499930->-5552841363973466708 + + + + + +-8584387631107793650 + +simple-shuffle-group + + + +-5552841363973466708->-8584387631107793650 + + + + + +8118300845161018722 + +19 + + + +8118300845161018722->7343217716023499930 + + + + + +6828692398800769550 + +getitem + + + +8118300845161018722->6828692398800769550 + + + + + +3913254328742431391 + +19 + + + +3913254328742431391->7343217716023499930 + + + + + +-5601875366903900661 + +assign + + + +-5552841363976714283 + +20 + + + +-5601875366903900661->-5552841363976714283 + + + + + +-5500450667769605209 + +simple-shuffle-group + + + +-5552841363976714283->-5500450667769605209 + + + + + +3913254328741348866 + +20 + + + +3913254328741348866->-5601875366903900661 + + + + + +8118300845162101247 + +20 + + + +8118300845162101247->-5601875366903900661 + + + + + +-8726089407451588441 + +getitem + + + +8118300845162101247->-8726089407451588441 + + + + + +-7815079351704980736 + +assign + + + +-5552841363975631758 + +21 + + + +-7815079351704980736->-5552841363975631758 + + + + + +-5332030766655223284 + +simple-shuffle-group + + + +-5552841363975631758->-5332030766655223284 + + + + + +8118300845163183772 + +21 + + + +8118300845163183772->-7815079351704980736 + + + + + +-8557629714311683316 + +getitem + + + +8118300845163183772->-8557629714311683316 + + + + + +3913254328740266341 + +21 + + + +3913254328740266341->-7815079351704980736 + + + + + +-4198537445511134911 + +assign + + + +-5552841363978879333 + +22 + + + +-4198537445511134911->-5552841363978879333 + + + + + +-1663411785097453859 + +simple-shuffle-group + + + +-5552841363978879333->-1663411785097453859 + + + + + +3913254328739183816 + +22 + + + +3913254328739183816->-4198537445511134911 + + + + + +8118300845164266297 + +22 + + + +8118300845164266297->-4198537445511134911 + + + + + +3010474121475855325 + +getitem + + + +8118300845164266297->3010474121475855325 + + + + + +4423486728879375830 + +assign + + + +-5552841363977796808 + +23 + + + +4423486728879375830->-5552841363977796808 + + + + + +6753518853319697266 + +simple-shuffle-group + + + +-5552841363977796808->6753518853319697266 + + + + + +3913254328738101291 + +23 + + + +3913254328738101291->4423486728879375830 + + + + + +8118300845165348822 + +23 + + + +8118300845165348822->4423486728879375830 + + + + + +1287114475866766450 + +getitem + + + +8118300845165348822->1287114475866766450 + + + + + +-1667540070124421545 + +assign + + + +-5552841363963723983 + +24 + + + +-1667540070124421545->-5552841363963723983 + + + + + +8470169458848030491 + +simple-shuffle-group + + + +-5552841363963723983->8470169458848030491 + + + + + +8118300845166431347 + +24 + + + +8118300845166431347->-1667540070124421545 + + + + + +-2444155538673773541 + +getitem + + + +8118300845166431347->-2444155538673773541 + + + + + +3913254328754339166 + +24 + + + +3913254328754339166->-1667540070124421545 + + + + + +-2790328412646946820 + +assign + + + +-5552841363962641458 + +25 + + + +-2790328412646946820->-5552841363962641458 + + + + + +7457217997785096016 + +simple-shuffle-group + + + +-5552841363962641458->7457217997785096016 + + + + + +8118300845167513872 + +25 + + + +8118300845167513872->-2790328412646946820 + + + + + +5898352828336635984 + +getitem + + + +8118300845167513872->5898352828336635984 + + + + + +3913254328753256641 + +25 + + + +3913254328753256641->-2790328412646946820 + + + + + +-786569163910162995 + +assign + + + +-5552841363965889033 + +26 + + + +-786569163910162995->-5552841363965889033 + + + + + +-5580019870367564975 + +simple-shuffle-group + + + +-5552841363965889033->-5580019870367564975 + + + + + +3913254328752174116 + +26 + + + +3913254328752174116->-786569163910162995 + + + + + +8118300845168596397 + +26 + + + +8118300845168596397->-786569163910162995 + + + + + +-1511108317547753391 + +getitem + + + +8118300845168596397->-1511108317547753391 + + + + + +8357844436385851746 + +assign + + + +-5552841363964806508 + +27 + + + +8357844436385851746->-5552841363964806508 + + + + + +1729916036795262950 + +simple-shuffle-group + + + +-5552841363964806508->1729916036795262950 + + + + + +8118300845169678922 + +27 + + + +8118300845169678922->8357844436385851746 + + + + + +-1343778321326723866 + +getitem + + + +8118300845169678922->-1343778321326723866 + + + + + +3913254328751091591 + +27 + + + +3913254328751091591->8357844436385851746 + + + + + +1493526254836469555 + +assign + + + +-5552841363968054083 + +28 + + + +1493526254836469555->-5552841363968054083 + + + + + +3638677798482865791 + +simple-shuffle-group + + + +-5552841363968054083->3638677798482865791 + + + + + +3913254328750009066 + +28 + + + +3913254328750009066->1493526254836469555 + + + + + +8118300845170761447 + +28 + + + +8118300845170761447->1493526254836469555 + + + + + +400862984985228159 + +getitem + + + +8118300845170761447->400862984985228159 + + + + + +388466323209051080 + +assign + + + +-5552841363966971558 + +29 + + + +388466323209051080->-5552841363966971558 + + + + + +2612727456218275716 + +simple-shuffle-group + + + +-5552841363966971558->2612727456218275716 + + + + + +8118300845171843972 + +29 + + + +8118300845171843972->388466323209051080 + + + + + +-722764826922933116 + +getitem + + + +8118300845171843972->-722764826922933116 + + + + + +3913254328748926541 + +29 + + + +3913254328748926541->388466323209051080 + + + + + +5248437884152712905 + +assign + + + +-5552841363970219133 + +30 + + + +5248437884152712905->-5552841363970219133 + + + + + +-2568180422635507275 + +simple-shuffle-group + + + +-5552841363970219133->-2568180422635507275 + + + + + +3913254328747844016 + +30 + + + +3913254328747844016->5248437884152712905 + + + + + +8118300845172926497 + +30 + + + +8118300845172926497->5248437884152712905 + + + + + +-6356707251664322891 + +getitem + + + +8118300845172926497->-6356707251664322891 + + + + + +2594171191527871230 + +assign + + + +-5552841363969136608 + +31 + + + +2594171191527871230->-5552841363969136608 + + + + + +-3738771995068067350 + +simple-shuffle-group + + + +-5552841363969136608->-3738771995068067350 + + + + + +3913254328746761491 + +31 + + + +3913254328746761491->2594171191527871230 + + + + + +8118300845174009022 + +31 + + + +8118300845174009022->2594171191527871230 + + + + + +-7946069717160102166 + +getitem + + + +8118300845174009022->-7946069717160102166 + + + + + +-8440087274343199105 + +assign + + + +-5552841363989704583 + +32 + + + +-8440087274343199105->-5552841363989704583 + + + + + +5813256369735053107 + +simple-shuffle-group + + + +-5552841363989704583->5813256369735053107 + + + + + +8118300845175091547 + +32 + + + +8118300845175091547->-8440087274343199105 + + + + + +-4132459013999282125 + +getitem + + + +8118300845175091547->-4132459013999282125 + + + + + +3913254328693717766 + +32 + + + +3913254328693717766->-8440087274343199105 + + + + + +8906279582036623236 + +assign + + + +-5552841363988622058 + +33 + + + +8906279582036623236->-5552841363988622058 + + + + + +5898408974926527432 + +simple-shuffle-group + + + +-5552841363988622058->5898408974926527432 + + + + + +8118300845176174072 + +33 + + + +8118300845176174072->8906279582036623236 + + + + + +-5146518870954803000 + +getitem + + + +8118300845176174072->-5146518870954803000 + + + + + +3913254328692635241 + +33 + + + +3913254328692635241->8906279582036623236 + + + + + +3102679448189227445 + +assign + + + +-5552841363991869633 + +34 + + + +3102679448189227445->-5552841363991869633 + + + + + +-8291772548664987959 + +simple-shuffle-group + + + +-5552841363991869633->-8291772548664987959 + + + + + +3913254328691552716 + +34 + + + +3913254328691552716->3102679448189227445 + + + + + +8118300845177256597 + +34 + + + +8118300845177256597->3102679448189227445 + + + + + +7063530977728055241 + +getitem + + + +8118300845177256597->7063530977728055241 + + + + + +1932087875756667370 + +assign + + + +-5552841363990787108 + +35 + + + +1932087875756667370->-5552841363990787108 + + + + + +-2682479808520719618 + +simple-shuffle-group + + + +-5552841363990787108->-2682479808520719618 + + + + + +8118300845178339122 + +35 + + + +8118300845178339122->1932087875756667370 + + + + + +5962901090108679166 + +getitem + + + +8118300845178339122->5962901090108679166 + + + + + +3913254328690470191 + +35 + + + +3913254328690470191->1932087875756667370 + + + + + +-4195855525028888805 + +assign + + + +-5552841363994034683 + +36 + + + +-4195855525028888805->-5552841363994034683 + + + + + +894390325600018007 + +simple-shuffle-group + + + +-5552841363994034683->894390325600018007 + + + + + +3913254328689387666 + +36 + + + +3913254328689387666->-4195855525028888805 + + + + + +8118300845179421647 + +36 + + + +8118300845179421647->-4195855525028888805 + + + + + +6593509673253672791 + +getitem + + + +8118300845179421647->6593509673253672791 + + + + + +-5318666240571900080 + +assign + + + +-5552841363992952158 + +37 + + + +-5318666240571900080->-5552841363992952158 + + + + + +-118580761502603268 + +simple-shuffle-group + + + +-5552841363992952158->-118580761502603268 + + + + + +8118300845180504172 + +37 + + + +8118300845180504172->-5318666240571900080 + + + + + +-4400232675056090884 + +getitem + + + +8118300845180504172->-4400232675056090884 + + + + + +3913254328688305141 + +37 + + + +3913254328688305141->-5318666240571900080 + + + + + +7890497312855710545 + +assign + + + +-5552841363996199733 + +38 + + + +7890497312855710545->-5552841363996199733 + + + + + +4797800327400273357 + +simple-shuffle-group + + + +-5552841363996199733->4797800327400273357 + + + + + +3913254328687222616 + +38 + + + +3913254328687222616->7890497312855710545 + + + + + +8118300845181586697 + +38 + + + +8118300845181586697->7890497312855710545 + + + + + +8865603522051329741 + +getitem + + + +8118300845181586697->8865603522051329741 + + + + + +6794296283074103270 + +assign + + + +-5552841363995117208 + +39 + + + +6794296283074103270->-5552841363995117208 + + + + + +-7526970341236815518 + +simple-shuffle-group + + + +-5552841363995117208->-7526970341236815518 + + + + + +3913254328686140091 + +39 + + + +3913254328686140091->6794296283074103270 + + + + + +8118300845182669222 + +39 + + + +8118300845182669222->6794296283074103270 + + + + + +7695279701075143266 + +getitem + + + +8118300845182669222->7695279701075143266 + + + + + +-736211507050908761 + +assign + + + +-5552841363981044383 + +40 + + + +-736211507050908761->-5552841363981044383 + + + + + +8816635300898960907 + +simple-shuffle-group + + + +-5552841363981044383->8816635300898960907 + + + + + +8118300845183751747 + +40 + + + +8118300845183751747->-736211507050908761 + + + + + +4876200448185323275 + +getitem + + + +8118300845183751747->4876200448185323275 + + + + + +3913254328702377966 + +40 + + + +3913254328702377966->-736211507050908761 + + + + + +-3022680871371970036 + +assign + + + +-5552841363979961858 + +41 + + + +-3022680871371970036->-5552841363979961858 + + + + + +6454654824373900032 + +simple-shuffle-group + + + +-5552841363979961858->6454654824373900032 + + + + + +8118300845184834272 + +41 + + + +8118300845184834272->-3022680871371970036 + + + + + +3854353726223616000 + +getitem + + + +8118300845184834272->3854353726223616000 + + + + + +3913254328701295441 + +41 + + + +3913254328701295441->-3022680871371970036 + + + + + +-6016910132550795555 + +assign + + + +-5552841363983209433 + +42 + + + +-6016910132550795555->-5552841363983209433 + + + + + +-6433497853406125759 + +simple-shuffle-group + + + +-5552841363983209433->-6433497853406125759 + + + + + +3913254328700212916 + +42 + + + +3913254328700212916->-6016910132550795555 + + + + + +8118300845185916797 + +42 + + + +8118300845185916797->-6016910132550795555 + + + + + +-2364590913078115775 + +getitem + + + +8118300845185916797->-2364590913078115775 + + + + + +-7122006416510860430 + +assign + + + +-5552841363982126908 + +43 + + + +-7122006416510860430->-5552841363982126908 + + + + + +1926760143565546966 + +simple-shuffle-group + + + +-5552841363982126908->1926760143565546966 + + + + + +8118300845186999322 + +43 + + + +8118300845186999322->-7122006416510860430 + + + + + +-3545181493160267050 + +getitem + + + +8118300845186999322->-3545181493160267050 + + + + + +3913254328699130391 + +43 + + + +3913254328699130391->-7122006416510860430 + + + + + +-4901283025365724861 + +assign + + + +-5552841363985374483 + +44 + + + +-4901283025365724861->-5552841363985374483 + + + + + +3975448567071868207 + +simple-shuffle-group + + + +-5552841363985374483->3975448567071868207 + + + + + +3913254328698047866 + +44 + + + +3913254328698047866->-4901283025365724861 + + + + + +8118300845188081847 + +44 + + + +8118300845188081847->-4901283025365724861 + + + + + +712031526441712175 + +getitem + + + +8118300845188081847->712031526441712175 + + + + + +-4811428118634570536 + +assign + + + +-5552841363984291958 + +45 + + + +-4811428118634570536->-5552841363984291958 + + + + + +2800444899605902132 + +simple-shuffle-group + + + +-5552841363984291958->2800444899605902132 + + + + + +8118300845189164372 + +45 + + + +8118300845189164372->-4811428118634570536 + + + + + +-314222866438269900 + +getitem + + + +8118300845189164372->-314222866438269900 + + + + + +3913254328696965341 + +45 + + + +3913254328696965341->-4811428118634570536 + + + + + +-1189065430910740455 + +assign + + + +-5552841363987539533 + +46 + + + +-1189065430910740455->-5552841363987539533 + + + + + +-3574868481938336859 + +simple-shuffle-group + + + +-5552841363987539533->-3574868481938336859 + + + + + +3913254328695882816 + +46 + + + +3913254328695882816->-1189065430910740455 + + + + + +8118300845190246897 + +46 + + + +8118300845190246897->-1189065430910740455 + + + + + +-6538704684109296475 + +getitem + + + +8118300845190246897->-6538704684109296475 + + + + + +-2290835469817266930 + +assign + + + +-5552841363986457008 + +47 + + + +-2290835469817266930->-5552841363986457008 + + + + + +-4601145275586085734 + +simple-shuffle-group + + + +-5552841363986457008->-4601145275586085734 + + + + + +3913254328694800291 + +47 + + + +3913254328694800291->-2290835469817266930 + + + + + +8118300845191329422 + +47 + + + +8118300845191329422->-2290835469817266930 + + + + + +-7713758049293429350 + +getitem + + + +8118300845191329422->-7713758049293429350 + + + + + +-7581906986439441521 + +assign + + + +-5552841364007024983 + +48 + + + +-7581906986439441521->-5552841364007024983 + + + + + +4802138345093437923 + +simple-shuffle-group + + + +-5552841364007024983->4802138345093437923 + + + + + +8118300845192411947 + +48 + + + +8118300845192411947->-7581906986439441521 + + + + + +-7321685812140314909 + +getitem + + + +8118300845192411947->-7321685812140314909 + + + + + +3913254328711038166 + +48 + + + +3913254328711038166->-7581906986439441521 + + + + + +1736073078773354337 + +read-parquet + + + +1736073078773354337->8118300845140450747 + + + + + +1763480936917737462 + +read-parquet + + + +1763480936917737462->8118300845141533272 + + + + + +662836035698514987 + +read-parquet + + + +662836035698514987->8118300845142615797 + + + + + +-1990022824122577888 + +read-parquet + + + +-1990022824122577888->8118300845143698322 + + + + + +-3096206199484859963 + +read-parquet + + + +-3096206199484859963->8118300845144780847 + + + + + +-3648566165697600838 + +read-parquet + + + +-3648566165697600838->8118300845145863372 + + + + + +3661368604138481487 + +read-parquet + + + +3661368604138481487->8118300845146945897 + + + + + +-6212924928425602988 + +read-parquet + + + +-6212924928425602988->8118300845148028422 + + + + + +-7335735646220266263 + +read-parquet + + + +-7335735646220266263->8118300845149110947 + + + + + +-8436401865110682338 + +read-parquet + + + +-8436401865110682338->8118300845150193472 + + + + + +-84981528765872813 + +read-parquet + + + +-84981528765872813->8118300845151275997 + + + + + +7292504516243697128 + +read-parquet + + + +7292504516243697128->8118300845152358522 + + + + + +6266513311137174253 + +read-parquet + + + +6266513311137174253->8118300845153441047 + + + + + +-6647346064500263038 + +read-parquet + + + +-6647346064500263038->8118300845154523572 + + + + + +660174962385142903 + +read-parquet + + + +660174962385142903->8118300845155606097 + + + + + +-352791548806361572 + +read-parquet + + + +-352791548806361572->8118300845156688622 + + + + + +8068836846242868753 + +read-parquet + + + +8068836846242868753->8118300845157771147 + + + + + +6897141515905152678 + +read-parquet + + + +6897141515905152678->8118300845158853672 + + + + + +-4171027696436655397 + +read-parquet + + + +-4171027696436655397->8118300845159936197 + + + + + +-5271657594032581872 + +read-parquet + + + +-5271657594032581872->8118300845161018722 + + + + + +3228507486418504053 + +read-parquet + + + +3228507486418504053->8118300845162101247 + + + + + +2675343238217310378 + +read-parquet + + + +2675343238217310378->8118300845163183772 + + + + + +-8339625564663842497 + +read-parquet + + + +-8339625564663842497->8118300845164266297 + + + + + +9006509898100139844 + +read-parquet + + + +9006509898100139844->8118300845165348822 + + + + + +-941198667924574247 + +read-parquet + + + +-941198667924574247->8118300845166431347 + + + + + +-2045986367515983922 + +read-parquet + + + +-2045986367515983922->8118300845167513872 + + + + + +4072410934509321219 + +read-parquet + + + +4072410934509321219->8118300845168596397 + + + + + +4165318659288968344 + +read-parquet + + + +4165318659288968344->8118300845169678922 + + + + + +-6482740224149998947 + +read-parquet + + + +-6482740224149998947->8118300845170761447 + + + + + +-7495424002483907022 + +read-parquet + + + +-7495424002483907022->8118300845171843972 + + + + + +1081165264297570919 + +read-parquet + + + +1081165264297570919->8118300845172926497 + + + + + +-93833898236277556 + +read-parquet + + + +-93833898236277556->8118300845174009022 + + + + + +7132816159997376769 + +read-parquet + + + +7132816159997376769->8118300845175091547 + + + + + +6032148812613619094 + +read-parquet + + + +6032148812613619094->8118300845176174072 + + + + + +-3763288943798518581 + +read-parquet + + + +-3763288943798518581->8118300845177256597 + + + + + +-4863956300084961856 + +read-parquet + + + +-4863956300084961856->8118300845178339122 + + + + + +1703054623582080869 + +read-parquet + + + +1703054623582080869->8118300845179421647 + + + + + +1870348286280418394 + +read-parquet + + + +1870348286280418394->8118300845180504172 + + + + + +8063387160640714735 + +read-parquet + + + +8063387160640714735->8118300845181586697 + + + + + +-1966404963579564556 + +read-parquet + + + +-1966404963579564556->8118300845182669222 + + + + + +-3142565551788077431 + +read-parquet + + + +-3142565551788077431->8118300845183751747 + + + + + +-4238766539411831106 + +read-parquet + + + +-4238766539411831106->8118300845184834272 + + + + + +3218665187786184435 + +read-parquet + + + +3218665187786184435->8118300845185916797 + + + + + +-6868661492303918456 + +read-parquet + + + +-6868661492303918456->8118300845186999322 + + + + + +-8477289330623081331 + +read-parquet + + + +-8477289330623081331->8118300845188081847 + + + + + +-1019878842427002590 + +read-parquet + + + +-1019878842427002590->8118300845189164372 + + + + + +7397020012882944535 + +read-parquet + + + +7397020012882944535->8118300845190246897 + + + + + +6221180210485468860 + +read-parquet + + + +6221180210485468860->8118300845191329422 + + + + + +5124942925885067185 + +read-parquet + + + +5124942925885067185->8118300845192411947 + + + + + +-2441231537478399971 + +0 + + + +7739510092502631059->-2441231537478399971 + + + + + +-3593681563978083084 + +set_partitions_pre + + + +-2441231537478399971->-3593681563978083084 + + + + + +-2441231537477317446 + +1 + + + +6638842773940018984->-2441231537477317446 + + + + + +-2567463503477646609 + +set_partitions_pre + + + +-2441231537477317446->-2567463503477646609 + + + + + +-2441231537480565021 + +2 + + + +1404753993089809193->-2441231537480565021 + + + + + +7302418999361554266 + +set_partitions_pre + + + +-2441231537480565021->7302418999361554266 + + + + + +-2441231537479482496 + +3 + + + +8836797070869143134->-2441231537479482496 + + + + + +713287161511293541 + +set_partitions_pre + + + +-2441231537479482496->713287161511293541 + + + + + +-2441231537474069871 + +4 + + + +2382585336333778359->-2441231537474069871 + + + + + +574894948871399616 + +set_partitions_pre + + + +-2441231537474069871->574894948871399616 + + + + + +-2441231537472987346 + +5 + + + +-7642776713948807332->-2441231537472987346 + + + + + +1597809244863826891 + +set_partitions_pre + + + +-2441231537472987346->1597809244863826891 + + + + + +-2441231537476234921 + +6 + + + +-3465338802196343507->-2441231537476234921 + + + + + +4329507112420694166 + +set_partitions_pre + + + +-2441231537476234921->4329507112420694166 + + + + + +-2441231537475152396 + +7 + + + +3405236447878263234->-2441231537475152396 + + + + + +5434598818252631041 + +set_partitions_pre + + + +-2441231537475152396->5434598818252631041 + + + + + +-2441231537487060171 + +8 + + + +682995203040558443->-2441231537487060171 + + + + + +-6640283182512226884 + +set_partitions_pre + + + +-2441231537487060171->-6640283182512226884 + + + + + +-2441231537485977646 + +9 + + + +-492026381288022432->-2441231537485977646 + + + + + +3389240123800010007 + +set_partitions_pre + + + +-2441231537485977646->3389240123800010007 + + + + + +-2441231537489225221 + +10 + + + +3238498460401228193->-2441231537489225221 + + + + + +4415516917447758882 + +set_partitions_pre + + + +-2441231537489225221->4415516917447758882 + + + + + +-2441231537488142696 + +11 + + + +1695960576957278518->-2441231537488142696 + + + + + +8921225344147458957 + +set_partitions_pre + + + +-2441231537488142696->8921225344147458957 + + + + + +-2441231537482730071 + +12 + + + +-5993672831596595057->-2441231537482730071 + + + + + +-8424873795105022584 + +set_partitions_pre + + + +-2441231537482730071->-8424873795105022584 + + + + + +-2441231537481647546 + +13 + + + +-7090997268567021932->-2441231537481647546 + + + + + +-7412190014554103309 + +set_partitions_pre + + + +-2441231537481647546->-7412190014554103309 + + + + + +-2441231537484895121 + +14 + + + +-1589350818223808507->-2441231537484895121 + + + + + +2457943514608978366 + +set_partitions_pre + + + +-2441231537484895121->2457943514608978366 + + + + + +-2441231537483812596 + +15 + + + +-3060854971146904582->-2441231537483812596 + + + + + +-4698777819117002359 + +set_partitions_pre + + + +-2441231537483812596->-4698777819117002359 + + + + + +-2441231537461079571 + +16 + + + +-4391383745347844541->-2441231537461079571 + + + + + +2725490314523202532 + +set_partitions_pre + + + +-2441231537461079571->2725490314523202532 + + + + + +-2441231537459997046 + +17 + + + +-5492850904186251816->-2441231537459997046 + + + + + +-5773603894127644193 + +set_partitions_pre + + + +-2441231537459997046->-5773603894127644193 + + + + + +-2441231537463244621 + +18 + + + +7854932871154028825->-2441231537463244621 + + + + + +-4668507681111937718 + +set_partitions_pre + + + +-2441231537463244621->-4668507681111937718 + + + + + +-2441231537462162096 + +19 + + + +6828692398800769550->-2441231537462162096 + + + + + +5846911347151355157 + +set_partitions_pre + + + +-2441231537462162096->5846911347151355157 + + + + + +-2441231537456749471 + +20 + + + +-8726089407451588441->-2441231537456749471 + + + + + +6868735730456250032 + +set_partitions_pre + + + +-2441231537456749471->6868735730456250032 + + + + + +-2441231537455666946 + +21 + + + +-8557629714311683316->-2441231537455666946 + + + + + +-1551787781658562693 + +set_partitions_pre + + + +-2441231537455666946->-1551787781658562693 + + + + + +-2441231537458914521 + +22 + + + +3010474121475855325->-2441231537458914521 + + + + + +-428996024886828218 + +set_partitions_pre + + + +-2441231537458914521->-428996024886828218 + + + + + +-2441231537457831996 + +23 + + + +1287114475866766450->-2441231537457831996 + + + + + +-7737624282625434959 + +set_partitions_pre + + + +-2441231537457831996->-7737624282625434959 + + + + + +-2441231537469739771 + +24 + + + +-2444155538673773541->-2441231537469739771 + + + + + +-7642846355923422868 + +set_partitions_pre + + + +-2441231537469739771->-7642846355923422868 + + + + + +-2441231537468657246 + +25 + + + +5898352828336635984->-2441231537468657246 + + + + + +3572800924827574023 + +set_partitions_pre + + + +-2441231537468657246->3572800924827574023 + + + + + +-2441231537471904821 + +26 + + + +-1511108317547753391->-2441231537471904821 + + + + + +4743392426315775698 + +set_partitions_pre + + + +-2441231537471904821->4743392426315775698 + + + + + +-2441231537470822296 + +27 + + + +-1343778321326723866->-2441231537470822296 + + + + + +-3201609325716463043 + +set_partitions_pre + + + +-2441231537470822296->-3201609325716463043 + + + + + +-2441231537465409671 + +28 + + + +400862984985228159->-2441231537465409671 + + + + + +-2101268431898234568 + +set_partitions_pre + + + +-2441231537465409671->-2101268431898234568 + + + + + +-2441231537464327146 + +29 + + + +-722764826922933116->-2441231537464327146 + + + + + +7919686029495490723 + +set_partitions_pre + + + +-2441231537464327146->7919686029495490723 + + + + + +-2441231537467574721 + +30 + + + +-6356707251664322891->-2441231537467574721 + + + + + +7756799921339660398 + +set_partitions_pre + + + +-2441231537467574721->7756799921339660398 + + + + + +-2441231537466492196 + +31 + + + +-7946069717160102166->-2441231537466492196 + + + + + +1625131513853550457 + +set_partitions_pre + + + +-2441231537466492196->1625131513853550457 + + + + + +-2441231537443759171 + +32 + + + +-4132459013999282125->-2441231537443759171 + + + + + +-772969863956435116 + +set_partitions_pre + + + +-2441231537443759171->-772969863956435116 + + + + + +-2441231537442676646 + +33 + + + +-5146518870954803000->-2441231537442676646 + + + + + +1331601206300372559 + +set_partitions_pre + + + +-2441231537442676646->1331601206300372559 + + + + + +-2441231537445924221 + +34 + + + +7063530977728055241->-2441231537445924221 + + + + + +3623583746518971834 + +set_partitions_pre + + + +-2441231537445924221->3623583746518971834 + + + + + +-2441231537444841696 + +35 + + + +5962901090108679166->-2441231537444841696 + + + + + +4728676588603789509 + +set_partitions_pre + + + +-2441231537444841696->4728676588603789509 + + + + + +-2441231537439429071 + +36 + + + +6593509673253672791->-2441231537439429071 + + + + + +-3767091401597653216 + +set_partitions_pre + + + +-2441231537439429071->-3767091401597653216 + + + + + +-2441231537438346546 + +37 + + + +-4400232675056090884->-2441231537438346546 + + + + + +6177180518828598059 + +set_partitions_pre + + + +-2441231537438346546->6177180518828598059 + + + + + +-2441231537441594121 + +38 + + + +8865603522051329741->-2441231537441594121 + + + + + +7264505796336127734 + +set_partitions_pre + + + +-2441231537441594121->7264505796336127734 + + + + + +-2441231537440511596 + +39 + + + +7695279701075143266->-2441231537440511596 + + + + + +-8965717188255983007 + +set_partitions_pre + + + +-2441231537440511596->-8965717188255983007 + + + + + +-2441231537452419371 + +40 + + + +4876200448185323275->-2441231537452419371 + + + + + +7450917841201033500 + +set_partitions_pre + + + +-2441231537452419371->7450917841201033500 + + + + + +-2441231537451336846 + +41 + + + +3854353726223616000->-2441231537451336846 + + + + + +8625921545213043575 + +set_partitions_pre + + + +-2441231537451336846->8625921545213043575 + + + + + +-2441231537454584421 + +42 + + + +-2364590913078115775->-2441231537454584421 + + + + + +-7845464889243127166 + +set_partitions_pre + + + +-2441231537454584421->-7845464889243127166 + + + + + +-2441231537453501896 + +43 + + + +-3545181493160267050->-2441231537453501896 + + + + + +-6744833867587881491 + +set_partitions_pre + + + +-2441231537453501896->-6744833867587881491 + + + + + +-2441231537448089271 + +44 + + + +712031526441712175->-2441231537448089271 + + + + + +-5638919354404584216 + +set_partitions_pre + + + +-2441231537448089271->-5638919354404584216 + + + + + +-2441231537447006746 + +45 + + + +-314222866438269900->-2441231537447006746 + + + + + +-2832549845120538541 + +set_partitions_pre + + + +-2441231537447006746->-2832549845120538541 + + + + + +-2441231537450254321 + +46 + + + +-6538704684109296475->-2441231537450254321 + + + + + +-2926615552573776866 + +set_partitions_pre + + + +-2441231537450254321->-2926615552573776866 + + + + + +-2441231537449171796 + +47 + + + +-7713758049293429350->-2441231537449171796 + + + + + +-1873637210231687191 + +set_partitions_pre + + + +-2441231537449171796->-1873637210231687191 + + + + + +-2441231537426438771 + +48 + + + +-7321685812140314909->-2441231537426438771 + + + + + +-4001387469308999100 + +set_partitions_pre + + + +-2441231537426438771->-4001387469308999100 + + + + + +-3593681563978083084->3913254328728358566 + + + + + +-2567463503477646609->3913254328727276041 + + + + + +7302418999361554266->3913254328726193516 + + + + + +713287161511293541->3913254328725110991 + + + + + +574894948871399616->3913254328724028466 + + + + + +1597809244863826891->3913254328722945941 + + + + + +4329507112420694166->3913254328721863416 + + + + + +5434598818252631041->3913254328720780891 + + + + + +-6640283182512226884->3913254328737018766 + + + + + +3389240123800010007->3913254328735936241 + + + + + +4415516917447758882->3913254328734853716 + + + + + +8921225344147458957->3913254328733771191 + + + + + +-8424873795105022584->3913254328732688666 + + + + + +-7412190014554103309->3913254328731606141 + + + + + +2457943514608978366->3913254328730523616 + + + + + +-4698777819117002359->3913254328729441091 + + + + + +2725490314523202532->3913254328745678966 + + + + + +-5773603894127644193->3913254328744596441 + + + + + +-4668507681111937718->3913254328743513916 + + + + + +5846911347151355157->3913254328742431391 + + + + + +6868735730456250032->3913254328741348866 + + + + + +-1551787781658562693->3913254328740266341 + + + + + +-428996024886828218->3913254328739183816 + + + + + +-7737624282625434959->3913254328738101291 + + + + + +-7642846355923422868->3913254328754339166 + + + + + +3572800924827574023->3913254328753256641 + + + + + +4743392426315775698->3913254328752174116 + + + + + +-3201609325716463043->3913254328751091591 + + + + + +-2101268431898234568->3913254328750009066 + + + + + +7919686029495490723->3913254328748926541 + + + + + +7756799921339660398->3913254328747844016 + + + + + +1625131513853550457->3913254328746761491 + + + + + +-772969863956435116->3913254328693717766 + + + + + +1331601206300372559->3913254328692635241 + + + + + +3623583746518971834->3913254328691552716 + + + + + +4728676588603789509->3913254328690470191 + + + + + +-3767091401597653216->3913254328689387666 + + + + + +6177180518828598059->3913254328688305141 + + + + + +7264505796336127734->3913254328687222616 + + + + + +-8965717188255983007->3913254328686140091 + + + + + +7450917841201033500->3913254328702377966 + + + + + +8625921545213043575->3913254328701295441 + + + + + +-7845464889243127166->3913254328700212916 + + + + + +-6744833867587881491->3913254328699130391 + + + + + +-5638919354404584216->3913254328698047866 + + + + + +-2832549845120538541->3913254328696965341 + + + + + +-2926615552573776866->3913254328695882816 + + + + + +-1873637210231687191->3913254328694800291 + + + + + +-4001387469308999100->3913254328711038166 + + + + + +-4202570559910840035 + +0 + + + +3561848877138230675->-4202570559910840035 + + + + + +1484894084834626618 + +simple-shuffle-split + + + +-4202570559910840035->1484894084834626618 + + + + + +-7550883156245396807 + +simple-shuffle-split + + + +-4202570559910840035->-7550883156245396807 + + + + + +7394493774267332684 + +simple-shuffle-split + + + +-4202570559910840035->7394493774267332684 + + + + + +8995032920700762459 + +simple-shuffle-split + + + +-4202570559910840035->8995032920700762459 + + + + + +-4119180910339601682 + +simple-shuffle-split + + + +-4202570559910840035->-4119180910339601682 + + + + + +7611195280567332109 + +simple-shuffle-split + + + +-4202570559910840035->7611195280567332109 + + + + + +1879494051309515984 + +simple-shuffle-split + + + +-4202570559910840035->1879494051309515984 + + + + + +4480203053389087759 + +simple-shuffle-split + + + +-4202570559910840035->4480203053389087759 + + + + + +2107302800818605618 + +simple-shuffle-split + + + +-4202570559910840035->2107302800818605618 + + + + + +3079740599528837393 + +simple-shuffle-split + + + +-4202570559910840035->3079740599528837393 + + + + + +1323560350830512868 + +simple-shuffle-split + + + +-4202570559910840035->1323560350830512868 + + + + + +1546817770183238643 + +simple-shuffle-split + + + +-4202570559910840035->1546817770183238643 + + + + + +-1217582318854381082 + +simple-shuffle-split + + + +-4202570559910840035->-1217582318854381082 + + + + + +-169756980801600507 + +simple-shuffle-split + + + +-4202570559910840035->-169756980801600507 + + + + + +-6577335906397208632 + +simple-shuffle-split + + + +-4202570559910840035->-6577335906397208632 + + + + + +-5525106383658944857 + +simple-shuffle-split + + + +-4202570559910840035->-5525106383658944857 + + + + + +-4666122521912370998 + +simple-shuffle-split + + + +-4202570559910840035->-4666122521912370998 + + + + + +-2916801110675116023 + +simple-shuffle-split + + + +-4202570559910840035->-2916801110675116023 + + + + + +-7206269399868576548 + +simple-shuffle-split + + + +-4202570559910840035->-7206269399868576548 + + + + + +-5639669815009597973 + +simple-shuffle-split + + + +-4202570559910840035->-5639669815009597973 + + + + + +8265622953955227518 + +simple-shuffle-split + + + +-4202570559910840035->8265622953955227518 + + + + + +-7501570253216124323 + +simple-shuffle-split + + + +-4202570559910840035->-7501570253216124323 + + + + + +5798739089968052768 + +simple-shuffle-split + + + +-4202570559910840035->5798739089968052768 + + + + + +7212286209266448543 + +simple-shuffle-split + + + +-4202570559910840035->7212286209266448543 + + + + + +-1663520206827355198 + +simple-shuffle-split + + + +-4202570559910840035->-1663520206827355198 + + + + + +-9174731103954879839 + +simple-shuffle-split + + + +-4202570559910840035->-9174731103954879839 + + + + + +5247423636651724852 + +simple-shuffle-split + + + +-4202570559910840035->5247423636651724852 + + + + + +5100549947611951427 + +simple-shuffle-split + + + +-4202570559910840035->5100549947611951427 + + + + + +-116816442221308298 + +simple-shuffle-split + + + +-4202570559910840035->-116816442221308298 + + + + + +2121253143600782677 + +simple-shuffle-split + + + +-4202570559910840035->2121253143600782677 + + + + + +-1502186499427375848 + +simple-shuffle-split + + + +-4202570559910840035->-1502186499427375848 + + + + + +-1090363736461681673 + +simple-shuffle-split + + + +-4202570559910840035->-1090363736461681673 + + + + + +4864772829922005594 + +simple-shuffle-split + + + +-4202570559910840035->4864772829922005594 + + + + + +-3582671365812353831 + +simple-shuffle-split + + + +-4202570559910840035->-3582671365812353831 + + + + + +-495020488695857556 + +simple-shuffle-split + + + +-4202570559910840035->-495020488695857556 + + + + + +-8963669792474626181 + +simple-shuffle-split + + + +-4202570559910840035->-8963669792474626181 + + + + + +-3083941708435505906 + +simple-shuffle-split + + + +-4202570559910840035->-3083941708435505906 + + + + + +-1471889485299179731 + +simple-shuffle-split + + + +-4202570559910840035->-1471889485299179731 + + + + + +5149482920558123760 + +simple-shuffle-split + + + +-4202570559910840035->5149482920558123760 + + + + + +5626409877918565935 + +simple-shuffle-split + + + +-4202570559910840035->5626409877918565935 + + + + + +4940929997580244050 + +simple-shuffle-split + + + +-4202570559910840035->4940929997580244050 + + + + + +6460692480019531569 + +simple-shuffle-split + + + +-4202570559910840035->6460692480019531569 + + + + + +2507884707565783044 + +simple-shuffle-split + + + +-4202570559910840035->2507884707565783044 + + + + + +3888695933329177875 + +simple-shuffle-split + + + +-4202570559910840035->3888695933329177875 + + + + + +-8996934411831836922 + +simple-shuffle-split + + + +-4202570559910840035->-8996934411831836922 + + + + + +-9222472477943357147 + +simple-shuffle-split + + + +-4202570559910840035->-9222472477943357147 + + + + + +-2046131356769234456 + +simple-shuffle-split + + + +-4202570559910840035->-2046131356769234456 + + + + + +7709348316417150919 + +simple-shuffle-split + + + +-4202570559910840035->7709348316417150919 + + + + + +5408852422873980650 + +simple-shuffle-split + + + +-4202570559910840035->5408852422873980650 + + + + + +-4202570559909757510 + +1 + + + +391848673754780200->-4202570559909757510 + + + + + +-6326953721752775749 + +simple-shuffle-split + + + +-4202570559909757510->-6326953721752775749 + + + + + +-8991661886735607124 + +simple-shuffle-split + + + +-4202570559909757510->-8991661886735607124 + + + + + +-3365412286419040999 + +simple-shuffle-split + + + +-4202570559909757510->-3365412286419040999 + + + + + +4680587689285289626 + +simple-shuffle-split + + + +-4202570559909757510->4680587689285289626 + + + + + +6560038413356320367 + +simple-shuffle-split + + + +-4202570559909757510->6560038413356320367 + + + + + +5516664421532978992 + +simple-shuffle-split + + + +-4202570559909757510->5516664421532978992 + + + + + +-8845895420737534099 + +simple-shuffle-split + + + +-4202570559909757510->-8845895420737534099 + + + + + +-375185646789188274 + +simple-shuffle-split + + + +-4202570559909757510->-375185646789188274 + + + + + +-5317347265349851565 + +simple-shuffle-split + + + +-4202570559909757510->-5317347265349851565 + + + + + +3313213723235198276 + +simple-shuffle-split + + + +-4202570559909757510->3313213723235198276 + + + + + +6275166469116070001 + +simple-shuffle-split + + + +-4202570559909757510->6275166469116070001 + + + + + +5231792478366593426 + +simple-shuffle-split + + + +-4202570559909757510->5231792478366593426 + + + + + +859273565801279751 + +simple-shuffle-split + + + +-4202570559909757510->859273565801279751 + + + + + +-2320928794648856024 + +simple-shuffle-split + + + +-4202570559909757510->-2320928794648856024 + + + + + +1197194994763788901 + +simple-shuffle-split + + + +-4202570559909757510->1197194994763788901 + + + + + +1903755951265555526 + +simple-shuffle-split + + + +-4202570559909757510->1903755951265555526 + + + + + +-3650015331882301365 + +simple-shuffle-split + + + +-4202570559909757510->-3650015331882301365 + + + + + +-3459926936677971140 + +simple-shuffle-split + + + +-4202570559909757510->-3459926936677971140 + + + + + +133565742423124585 + +simple-shuffle-split + + + +-4202570559909757510->133565742423124585 + + + + + +-2606641376605003990 + +simple-shuffle-split + + + +-4202570559909757510->-2606641376605003990 + + + + + +-6829538520669144065 + +simple-shuffle-split + + + +-4202570559909757510->-6829538520669144065 + + + + + +-7872912510344755840 + +simple-shuffle-split + + + +-4202570559909757510->-7872912510344755840 + + + + + +-4231490635433749315 + +simple-shuffle-split + + + +-4202570559909757510->-4231490635433749315 + + + + + +-6549489177168646690 + +simple-shuffle-split + + + +-4202570559909757510->-6549489177168646690 + + + + + +8773609046835760035 + +simple-shuffle-split + + + +-4202570559909757510->8773609046835760035 + + + + + +8442008876282599060 + +simple-shuffle-split + + + +-4202570559909757510->8442008876282599060 + + + + + +-8304391524974527231 + +simple-shuffle-split + + + +-4202570559909757510->-8304391524974527231 + + + + + +-8599728700690429406 + +simple-shuffle-split + + + +-4202570559909757510->-8599728700690429406 + + + + + +3095107204309032535 + +simple-shuffle-split + + + +-4202570559909757510->3095107204309032535 + + + + + +1618043341579447160 + +simple-shuffle-split + + + +-4202570559909757510->1618043341579447160 + + + + + +7389775948215132085 + +simple-shuffle-split + + + +-4202570559909757510->7389775948215132085 + + + + + +4142911239066551510 + +simple-shuffle-split + + + +-4202570559909757510->4142911239066551510 + + + + + +-7367348890869690981 + +simple-shuffle-split + + + +-4202570559909757510->-7367348890869690981 + + + + + +1101032660309557644 + +simple-shuffle-split + + + +-4202570559909757510->1101032660309557644 + + + + + +2810502952843640569 + +simple-shuffle-split + + + +-4202570559909757510->2810502952843640569 + + + + + +3075022808948815994 + +simple-shuffle-split + + + +-4202570559909757510->3075022808948815994 + + + + + +-5711058139326396081 + +simple-shuffle-split + + + +-4202570559909757510->-5711058139326396081 + + + + + +-6755541633296271856 + +simple-shuffle-split + + + +-4202570559909757510->-6755541633296271856 + + + + + +-364572285135697331 + +simple-shuffle-split + + + +-4202570559909757510->-364572285135697331 + + + + + +-1400213574783537106 + +simple-shuffle-split + + + +-4202570559909757510->-1400213574783537106 + + + + + +-4596449760938753997 + +simple-shuffle-split + + + +-4202570559909757510->-4596449760938753997 + + + + + +2271746485097355044 + +simple-shuffle-split + + + +-4202570559909757510->2271746485097355044 + + + + + +-3212615526117206447 + +simple-shuffle-split + + + +-4202570559909757510->-3212615526117206447 + + + + + +-3544215734376878222 + +simple-shuffle-split + + + +-4202570559909757510->-3544215734376878222 + + + + + +-2927905435481088281 + +simple-shuffle-split + + + +-4202570559909757510->-2927905435481088281 + + + + + +3853878086445070344 + +simple-shuffle-split + + + +-4202570559909757510->3853878086445070344 + + + + + +-8877775409583625147 + +simple-shuffle-split + + + +-4202570559909757510->-8877775409583625147 + + + + + +8082992868573344294 + +simple-shuffle-split + + + +-4202570559909757510->8082992868573344294 + + + + + +6021556619709705003 + +simple-shuffle-split + + + +-4202570559909757510->6021556619709705003 + + + + + +-4202570559913005085 + +2 + + + +4648106040364701225->-4202570559913005085 + + + + + +-7153607758011772404 + +simple-shuffle-split + + + +-4202570559913005085->-7153607758011772404 + + + + + +2671739413575161371 + +simple-shuffle-split + + + +-4202570559913005085->2671739413575161371 + + + + + +-5310255680497480454 + +simple-shuffle-split + + + +-4202570559913005085->-5310255680497480454 + + + + + +-4274668558010425479 + +simple-shuffle-split + + + +-4202570559913005085->-4274668558010425479 + + + + + +3397640077024762512 + +simple-shuffle-split + + + +-4202570559913005085->3397640077024762512 + + + + + +4370047218869479887 + +simple-shuffle-split + + + +-4202570559913005085->4370047218869479887 + + + + + +-1355645835306697554 + +simple-shuffle-split + + + +-4202570559913005085->-1355645835306697554 + + + + + +8616932943226245837 + +simple-shuffle-split + + + +-4202570559913005085->8616932943226245837 + + + + + +2328458508666252196 + +simple-shuffle-split + + + +-4202570559913005085->2328458508666252196 + + + + + +-3970589522996830029 + +simple-shuffle-split + + + +-4202570559913005085->-3970589522996830029 + + + + + +-2918356558383319054 + +simple-shuffle-split + + + +-4202570559913005085->-2918356558383319054 + + + + + +-187003906035455279 + +simple-shuffle-split + + + +-4202570559913005085->-187003906035455279 + + + + + +-3302229621990768504 + +simple-shuffle-split + + + +-4202570559913005085->-3302229621990768504 + + + + + +36405955408999271 + +simple-shuffle-split + + + +-4202570559913005085->36405955408999271 + + + + + +-282609514560785754 + +simple-shuffle-split + + + +-4202570559913005085->-282609514560785754 + + + + + +213584497347278021 + +simple-shuffle-split + + + +-4202570559913005085->213584497347278021 + + + + + +5776795197444707996 + +simple-shuffle-split + + + +-4202570559913005085->5776795197444707996 + + + + + +6851346204147859371 + +simple-shuffle-split + + + +-4202570559913005085->6851346204147859371 + + + + + +-8920298307025796470 + +simple-shuffle-split + + + +-4202570559913005085->-8920298307025796470 + + + + + +-8615837915483575095 + +simple-shuffle-split + + + +-4202570559913005085->-8615837915483575095 + + + + + +1362969017100182496 + +simple-shuffle-split + + + +-4202570559913005085->1362969017100182496 + + + + + +2262125795259096671 + +simple-shuffle-split + + + +-4202570559913005085->2262125795259096671 + + + + + +1552128932466378046 + +simple-shuffle-split + + + +-4202570559913005085->1552128932466378046 + + + + + +3799192399535675421 + +simple-shuffle-split + + + +-4202570559913005085->3799192399535675421 + + + + + +-2048179483005988620 + +simple-shuffle-split + + + +-4202570559913005085->-2048179483005988620 + + + + + +-361754552017269245 + +simple-shuffle-split + + + +-4202570559913005085->-361754552017269245 + + + + + +562870606765491330 + +simple-shuffle-split + + + +-4202570559913005085->562870606765491330 + + + + + +2489933337640228705 + +simple-shuffle-split + + + +-4202570559913005085->2489933337640228705 + + + + + +-7150234107347745320 + +simple-shuffle-split + + + +-4202570559913005085->-7150234107347745320 + + + + + +-6114609515189039945 + +simple-shuffle-split + + + +-4202570559913005085->-6114609515189039945 + + + + + +-5063387495419885770 + +simple-shuffle-split + + + +-4202570559913005085->-5063387495419885770 + + + + + +-2955086794830767595 + +simple-shuffle-split + + + +-4202570559913005085->-2955086794830767595 + + + + + +2150194437379193644 + +simple-shuffle-split + + + +-4202570559913005085->2150194437379193644 + + + + + +-4658566535144544197 + +simple-shuffle-split + + + +-4202570559913005085->-4658566535144544197 + + + + + +-3610628689385173222 + +simple-shuffle-split + + + +-4202570559913005085->-3610628689385173222 + + + + + +-3234273388893510247 + +simple-shuffle-split + + + +-4202570559913005085->-3234273388893510247 + + + + + +-1697612162603083856 + +simple-shuffle-split + + + +-4202570559913005085->-1697612162603083856 + + + + + +8118570929155131119 + +simple-shuffle-split + + + +-4202570559913005085->8118570929155131119 + + + + + +-7531293356333532722 + +simple-shuffle-split + + + +-4202570559913005085->-7531293356333532722 + + + + + +-7673715777865084947 + +simple-shuffle-split + + + +-4202570559913005085->-7673715777865084947 + + + + + +4926497008895487428 + +simple-shuffle-split + + + +-4202570559913005085->4926497008895487428 + + + + + +7089148444044846803 + +simple-shuffle-split + + + +-4202570559913005085->7089148444044846803 + + + + + +6459111210588751378 + +simple-shuffle-split + + + +-4202570559913005085->6459111210588751378 + + + + + +-2164862795734232847 + +simple-shuffle-split + + + +-4202570559913005085->-2164862795734232847 + + + + + +6734962443719282344 + +simple-shuffle-split + + + +-4202570559913005085->6734962443719282344 + + + + + +2125483606212852103 + +simple-shuffle-split + + + +-4202570559913005085->2125483606212852103 + + + + + +-6352023500899998522 + +simple-shuffle-split + + + +-4202570559913005085->-6352023500899998522 + + + + + +-4881972920246155547 + +simple-shuffle-split + + + +-4202570559913005085->-4881972920246155547 + + + + + +-2247717051537387588 + +simple-shuffle-split + + + +-4202570559913005085->-2247717051537387588 + + + + + +-4202570559911922560 + +3 + + + +4658027425283123550->-4202570559911922560 + + + + + +-4972632081319006931 + +simple-shuffle-split + + + +-4202570559911922560->-4972632081319006931 + + + + + +-4782560445697445106 + +simple-shuffle-split + + + +-4202570559911922560->-4782560445697445106 + + + + + +-9093401613779145681 + +simple-shuffle-split + + + +-4202570559911922560->-9093401613779145681 + + + + + +8308860109938309360 + +simple-shuffle-split + + + +-4202570559911922560->8308860109938309360 + + + + + +-423330542066881831 + +simple-shuffle-split + + + +-4202570559911922560->-423330542066881831 + + + + + +8133971699876450394 + +simple-shuffle-split + + + +-4202570559911922560->8133971699876450394 + + + + + +7583795547091185019 + +simple-shuffle-split + + + +-4202570559911922560->7583795547091185019 + + + + + +-2171456135574231956 + +simple-shuffle-split + + + +-4202570559911922560->-2171456135574231956 + + + + + +7525656288220581669 + +simple-shuffle-split + + + +-4202570559911922560->7525656288220581669 + + + + + +4849430470623737094 + +simple-shuffle-split + + + +-4202570559911922560->4849430470623737094 + + + + + +-5197520953667965497 + +simple-shuffle-split + + + +-4202570559911922560->-5197520953667965497 + + + + + +2303458327306631144 + +simple-shuffle-split + + + +-4202570559911922560->2303458327306631144 + + + + + +-6485099818142743247 + +simple-shuffle-split + + + +-4202570559911922560->-6485099818142743247 + + + + + +-8077889546049621422 + +simple-shuffle-split + + + +-4202570559911922560->-8077889546049621422 + + + + + +-7868938592294722797 + +simple-shuffle-split + + + +-4202570559911922560->-7868938592294722797 + + + + + +-1149231197722302972 + +simple-shuffle-split + + + +-4202570559911922560->-1149231197722302972 + + + + + +-1363796001833846147 + +simple-shuffle-split + + + +-4202570559911922560->-1363796001833846147 + + + + + +-3607404028595789922 + +simple-shuffle-split + + + +-4202570559911922560->-3607404028595789922 + + + + + +-4584271820288165697 + +simple-shuffle-split + + + +-4202570559911922560->-4584271820288165697 + + + + + +4441541530634315328 + +simple-shuffle-split + + + +-4202570559911922560->4441541530634315328 + + + + + +5712573200678899753 + +simple-shuffle-split + + + +-4202570559911922560->5712573200678899753 + + + + + +4363457285501830378 + +simple-shuffle-split + + + +-4202570559911922560->4363457285501830378 + + + + + +1487887347367393803 + +simple-shuffle-split + + + +-4202570559911922560->1487887347367393803 + + + + + +1752407194656485628 + +simple-shuffle-split + + + +-4202570559911922560->1752407194656485628 + + + + + +-8263668584995597963 + +simple-shuffle-split + + + +-4202570559911922560->-8263668584995597963 + + + + + +8768407791115709078 + +simple-shuffle-split + + + +-4202570559911922560->8768407791115709078 + + + + + +5962798397107553303 + +simple-shuffle-split + + + +-4202570559911922560->5962798397107553303 + + + + + +4998146337127177528 + +simple-shuffle-split + + + +-4202570559911922560->4998146337127177528 + + + + + +-2674162323368998463 + +simple-shuffle-split + + + +-4202570559911922560->-2674162323368998463 + + + + + +-4210702340188977438 + +simple-shuffle-split + + + +-4202570559911922560->-4210702340188977438 + + + + + +-4536068366197158813 + +simple-shuffle-split + + + +-4202570559911922560->-4536068366197158813 + + + + + +-7211166794714505388 + +simple-shuffle-split + + + +-4202570559911922560->-7211166794714505388 + + + + + +1879584913098746637 + +simple-shuffle-split + + + +-4202570559911922560->1879584913098746637 + + + + + +1393391115968231662 + +simple-shuffle-split + + + +-4202570559911922560->1393391115968231662 + + + + + +53130768217487887 + +simple-shuffle-split + + + +-4202570559911922560->53130768217487887 + + + + + +-1617370090724445488 + +simple-shuffle-split + + + +-4202570559911922560->-1617370090724445488 + + + + + +7947298445614999737 + +simple-shuffle-split + + + +-4202570559911922560->7947298445614999737 + + + + + +8212926658780424762 + +simple-shuffle-split + + + +-4202570559911922560->8212926658780424762 + + + + + +5095615094179428187 + +simple-shuffle-split + + + +-4202570559911922560->5095615094179428187 + + + + + +2853132207269838412 + +simple-shuffle-split + + + +-4202570559911922560->2853132207269838412 + + + + + +-6011189177384195579 + +simple-shuffle-split + + + +-4202570559911922560->-6011189177384195579 + + + + + +-6983649343589705754 + +simple-shuffle-split + + + +-4202570559911922560->-6983649343589705754 + + + + + +-6204794684623338329 + +simple-shuffle-split + + + +-4202570559911922560->-6204794684623338329 + + + + + +1301986149686189512 + +simple-shuffle-split + + + +-4202570559911922560->1301986149686189512 + + + + + +-7597539544118692079 + +simple-shuffle-split + + + +-4202570559911922560->-7597539544118692079 + + + + + +-750648501137839054 + +simple-shuffle-split + + + +-4202570559911922560->-750648501137839054 + + + + + +-1729866935177033229 + +simple-shuffle-split + + + +-4202570559911922560->-1729866935177033229 + + + + + +-4333622006463199004 + +simple-shuffle-split + + + +-4202570559911922560->-4333622006463199004 + + + + + +-3008319126361884579 + +simple-shuffle-split + + + +-4202570559911922560->-3008319126361884579 + + + + + +-4202570559906509935 + +4 + + + +-3373206642788804425->-4202570559906509935 + + + + + +-5766172080440701586 + +simple-shuffle-split + + + +-4202570559906509935->-5766172080440701586 + + + + + +-6523831002617041011 + +simple-shuffle-split + + + +-4202570559906509935->-6523831002617041011 + + + + + +8534707863943084880 + +simple-shuffle-split + + + +-4202570559906509935->8534707863943084880 + + + + + +2708015591099229839 + +simple-shuffle-split + + + +-4202570559906509935->2708015591099229839 + + + + + +7004998100991881914 + +simple-shuffle-split + + + +-4202570559906509935->7004998100991881914 + + + + + +-1406977365211904711 + +simple-shuffle-split + + + +-4202570559906509935->-1406977365211904711 + + + + + +-4361143243973501236 + +simple-shuffle-split + + + +-4202570559906509935->-4361143243973501236 + + + + + +5465043459040741339 + +simple-shuffle-split + + + +-4202570559906509935->5465043459040741339 + + + + + +4306986957390682214 + +simple-shuffle-split + + + +-4202570559906509935->4306986957390682214 + + + + + +6541740577366304389 + +simple-shuffle-split + + + +-4202570559906509935->6541740577366304389 + + + + + +-6025279943892340152 + +simple-shuffle-split + + + +-4202570559906509935->-6025279943892340152 + + + + + +3952815953405958439 + +simple-shuffle-split + + + +-4202570559906509935->3952815953405958439 + + + + + +8250103647880590514 + +simple-shuffle-split + + + +-4202570559906509935->8250103647880590514 + + + + + +-7540791197038294127 + +simple-shuffle-split + + + +-4202570559906509935->-7540791197038294127 + + + + + +-1510450120903569052 + +simple-shuffle-split + + + +-4202570559906509935->-1510450120903569052 + + + + + +8387850645911668339 + +simple-shuffle-split + + + +-4202570559906509935->8387850645911668339 + + + + + +-3530337333358067202 + +simple-shuffle-split + + + +-4202570559906509935->-3530337333358067202 + + + + + +-350113698468456227 + +simple-shuffle-split + + + +-4202570559906509935->-350113698468456227 + + + + + +3457890104974041248 + +simple-shuffle-split + + + +-4202570559906509935->3457890104974041248 + + + + + +-4574820826254078177 + +simple-shuffle-split + + + +-4202570559906509935->-4574820826254078177 + + + + + +2099376246823853898 + +simple-shuffle-split + + + +-4202570559906509935->2099376246823853898 + + + + + +-6522289507553159543 + +simple-shuffle-split + + + +-4202570559906509935->-6522289507553159543 + + + + + +694369751230477148 + +simple-shuffle-split + + + +-4202570559906509935->694369751230477148 + + + + + +1200219479784636523 + +simple-shuffle-split + + + +-4202570559906509935->1200219479784636523 + + + + + +-9082321085600011018 + +simple-shuffle-split + + + +-4202570559906509935->-9082321085600011018 + + + + + +-8038925783830374443 + +simple-shuffle-split + + + +-4202570559906509935->-8038925783830374443 + + + + + +4017953867954020248 + +simple-shuffle-split + + + +-4202570559906509935->4017953867954020248 + + + + + +5065757902676898423 + +simple-shuffle-split + + + +-4202570559906509935->5065757902676898423 + + + + + +-4646311807164342718 + +simple-shuffle-split + + + +-4202570559906509935->-4646311807164342718 + + + + + +-4864121707070635743 + +simple-shuffle-split + + + +-4202570559906509935->-4864121707070635743 + + + + + +-7679215214846478668 + +simple-shuffle-split + + + +-4202570559906509935->-7679215214846478668 + + + + + +-5445592398147602493 + +simple-shuffle-split + + + +-4202570559906509935->-5445592398147602493 + + + + + +-4737988174649712754 + +simple-shuffle-split + + + +-4202570559906509935->-4737988174649712754 + + + + + +-4880416271729936979 + +simple-shuffle-split + + + +-4202570559906509935->-4880416271729936979 + + + + + +-8870532717952975504 + +simple-shuffle-split + + + +-4202570559906509935->-8870532717952975504 + + + + + +-7386091407961800529 + +simple-shuffle-split + + + +-4202570559906509935->-7386091407961800529 + + + + + +830191255370134746 + +simple-shuffle-split + + + +-4202570559906509935->830191255370134746 + + + + + +-7692747281935040679 + +simple-shuffle-split + + + +-4202570559906509935->-7692747281935040679 + + + + + +-1763140995202902804 + +simple-shuffle-split + + + +-4202570559906509935->-1763140995202902804 + + + + + +479340721124773371 + +simple-shuffle-split + + + +-4202570559906509935->479340721124773371 + + + + + +5966892323893319046 + +simple-shuffle-split + + + +-4202570559906509935->5966892323893319046 + + + + + +-1939623493130329179 + +simple-shuffle-split + + + +-4202570559906509935->-1939623493130329179 + + + + + +4011876899052270696 + +simple-shuffle-split + + + +-4202570559906509935->4011876899052270696 + + + + + +4919088262600947271 + +simple-shuffle-split + + + +-4202570559906509935->4919088262600947271 + + + + + +2074188407509403346 + +simple-shuffle-split + + + +-4202570559906509935->2074188407509403346 + + + + + +-3752540159990166095 + +simple-shuffle-split + + + +-4202570559906509935->-3752540159990166095 + + + + + +7867737022545227396 + +simple-shuffle-split + + + +-4202570559906509935->7867737022545227396 + + + + + +-7902799139345038445 + +simple-shuffle-split + + + +-4202570559906509935->-7902799139345038445 + + + + + +7625060126592854046 + +simple-shuffle-split + + + +-4202570559906509935->7625060126592854046 + + + + + +-4202570559905427410 + +5 + + + +-4479352522949166500->-4202570559905427410 + + + + + +4692831239999388911 + +simple-shuffle-split + + + +-4202570559905427410->4692831239999388911 + + + + + +-6866373309042461264 + +simple-shuffle-split + + + +-4202570559905427410->-6866373309042461264 + + + + + +-2289027599731397139 + +simple-shuffle-split + + + +-4202570559905427410->-2289027599731397139 + + + + + +5666342200717070286 + +simple-shuffle-split + + + +-4202570559905427410->5666342200717070286 + + + + + +1954907472151383611 + +simple-shuffle-split + + + +-4202570559905427410->1954907472151383611 + + + + + +-1170154662713482964 + +simple-shuffle-split + + + +-4202570559905427410->-1170154662713482964 + + + + + +4456096072712817561 + +simple-shuffle-split + + + +-4202570559905427410->4456096072712817561 + + + + + +1809137598641736986 + +simple-shuffle-split + + + +-4202570559905427410->1809137598641736986 + + + + + +6918549934797842311 + +simple-shuffle-split + + + +-4202570559905427410->6918549934797842311 + + + + + +-2983071401574222680 + +simple-shuffle-split + + + +-4202570559905427410->-2983071401574222680 + + + + + +8451186549452434661 + +simple-shuffle-split + + + +-4202570559905427410->8451186549452434661 + + + + + +6288482063355468486 + +simple-shuffle-split + + + +-4202570559905427410->6288482063355468486 + + + + + +-6943402656622703405 + +simple-shuffle-split + + + +-4202570559905427410->-6943402656622703405 + + + + + +1535212815292615620 + +simple-shuffle-split + + + +-4202570559905427410->1535212815292615620 + + + + + +5557883608910114545 + +simple-shuffle-split + + + +-4202570559905427410->5557883608910114545 + + + + + +-4258340346833179630 + +simple-shuffle-split + + + +-4202570559905427410->-4258340346833179630 + + + + + +-1384098900173035905 + +simple-shuffle-split + + + +-4202570559905427410->-1384098900173035905 + + + + + +-2431902900566881280 + +simple-shuffle-split + + + +-4202570559905427410->-2431902900566881280 + + + + + +-7293376678730802371 + +simple-shuffle-split + + + +-4202570559905427410->-7293376678730802371 + + + + + +-410283888839962530 + +simple-shuffle-split + + + +-4202570559905427410->-410283888839962530 + + + + + +4171491891031317195 + +simple-shuffle-split + + + +-4202570559905427410->4171491891031317195 + + + + + +2604924088136396220 + +simple-shuffle-split + + + +-4202570559905427410->2604924088136396220 + + + + + +7150787558806925545 + +simple-shuffle-split + + + +-4202570559905427410->7150787558806925545 + + + + + +5215974249926099370 + +simple-shuffle-split + + + +-4202570559905427410->5215974249926099370 + + + + + +-7516804374939175721 + +simple-shuffle-split + + + +-4202570559905427410->-7516804374939175721 + + + + + +8195266512355184120 + +simple-shuffle-split + + + +-4202570559905427410->8195266512355184120 + + + + + +-5067652299550272971 + +simple-shuffle-split + + + +-4202570559905427410->-5067652299550272971 + + + + + +-5965723049242597546 + +simple-shuffle-split + + + +-4202570559905427410->-5965723049242597546 + + + + + +-3535038096713862621 + +simple-shuffle-split + + + +-4202570559905427410->-3535038096713862621 + + + + + +-5697725855374722796 + +simple-shuffle-split + + + +-4202570559905427410->-5697725855374722796 + + + + + +-1673133125810617471 + +simple-shuffle-split + + + +-4202570559905427410->-1673133125810617471 + + + + + +-1526258292810385246 + +simple-shuffle-split + + + +-4202570559905427410->-1526258292810385246 + + + + + +3691090192405782479 + +simple-shuffle-split + + + +-4202570559905427410->3691090192405782479 + + + + + +2202182557052917904 + +simple-shuffle-split + + + +-4202570559905427410->2202182557052917904 + + + + + +5002051313527476429 + +simple-shuffle-split + + + +-4202570559905427410->5002051313527476429 + + + + + +4041940690070459054 + +simple-shuffle-split + + + +-4202570559905427410->4041940690070459054 + + + + + +8166286810716746779 + +simple-shuffle-split + + + +-4202570559905427410->8166286810716746779 + + + + + +7240237983001915404 + +simple-shuffle-split + + + +-4202570559905427410->7240237983001915404 + + + + + +-7814696393882327687 + +simple-shuffle-split + + + +-4202570559905427410->-7814696393882327687 + + + + + +9064338590552738554 + +simple-shuffle-split + + + +-4202570559905427410->9064338590552738554 + + + + + +-4839867122395446937 + +simple-shuffle-split + + + +-4202570559905427410->-4839867122395446937 + + + + + +-6766893538626728312 + +simple-shuffle-split + + + +-4202570559905427410->-6766893538626728312 + + + + + +7942875876308153029 + +simple-shuffle-split + + + +-4202570559905427410->7942875876308153029 + + + + + +-3787633058144380762 + +simple-shuffle-split + + + +-4202570559905427410->-3787633058144380762 + + + + + +1467256573327312563 + +simple-shuffle-split + + + +-4202570559905427410->1467256573327312563 + + + + + +493709288006945188 + +simple-shuffle-split + + + +-4202570559905427410->493709288006945188 + + + + + +2851090807040354513 + +simple-shuffle-split + + + +-4202570559905427410->2851090807040354513 + + + + + +1877543522811172338 + +simple-shuffle-split + + + +-4202570559905427410->1877543522811172338 + + + + + +4791816337981140063 + +simple-shuffle-split + + + +-4202570559905427410->4791816337981140063 + + + + + +-4202570559908674985 + +6 + + + +-1851365200229048275->-4202570559908674985 + + + + + +-7227592232223727344 + +simple-shuffle-split + + + +-4202570559908674985->-7227592232223727344 + + + + + +3712333548520420431 + +simple-shuffle-split + + + +-4202570559908674985->3712333548520420431 + + + + + +4756817040307925806 + +simple-shuffle-split + + + +-4202570559908674985->4756817040307925806 + + + + + +-2063448707940247219 + +simple-shuffle-split + + + +-4202570559908674985->-2063448707940247219 + + + + + +7608553450364041356 + +simple-shuffle-split + + + +-4202570559908674985->7608553450364041356 + + + + + +971256081963288731 + +simple-shuffle-split + + + +-4202570559908674985->971256081963288731 + + + + + +754554506970582906 + +simple-shuffle-split + + + +-4202570559908674985->754554506970582906 + + + + + +2337344029002597881 + +simple-shuffle-split + + + +-4202570559908674985->2337344029002597881 + + + + + +-5172758483369357560 + +simple-shuffle-split + + + +-4202570559908674985->-5172758483369357560 + + + + + +-4673260690308380185 + +simple-shuffle-split + + + +-4202570559908674985->-4673260690308380185 + + + + + +5159508453385844006 + +simple-shuffle-split + + + +-4202570559908674985->5159508453385844006 + + + + + +7393149103809414981 + +simple-shuffle-split + + + +-4202570559908674985->7393149103809414981 + + + + + +-7909711097203421660 + +simple-shuffle-split + + + +-4202570559908674985->-7909711097203421660 + + + + + +-8127520997127035085 + +simple-shuffle-split + + + +-4202570559908674985->-8127520997127035085 + + + + + +-5316378847773530510 + +simple-shuffle-split + + + +-4202570559908674985->-5316378847773530510 + + + + + +5619158861487894865 + +simple-shuffle-split + + + +-4202570559908674985->5619158861487894865 + + + + + +4971332213787231840 + +simple-shuffle-split + + + +-4202570559908674985->4971332213787231840 + + + + + +-3573785981968170785 + +simple-shuffle-split + + + +-4202570559908674985->-3573785981968170785 + + + + + +-845893320343148610 + +simple-shuffle-split + + + +-4202570559908674985->-845893320343148610 + + + + + +-8348227419910455651 + +simple-shuffle-split + + + +-4202570559908674985->-8348227419910455651 + + + + + +1621008368414074140 + +simple-shuffle-split + + + +-4202570559908674985->1621008368414074140 + + + + + +4397107079729833515 + +simple-shuffle-split + + + +-4202570559908674985->4397107079729833515 + + + + + +-4303280582103022326 + +simple-shuffle-split + + + +-4202570559908674985->-4303280582103022326 + + + + + +6241526678049256265 + +simple-shuffle-split + + + +-4202570559908674985->6241526678049256265 + + + + + +7218399053394967640 + +simple-shuffle-split + + + +-4202570559908674985->7218399053394967640 + + + + + +8811184276335087415 + +simple-shuffle-split + + + +-4202570559908674985->8811184276335087415 + + + + + +1291081524850421174 + +simple-shuffle-split + + + +-4202570559908674985->1291081524850421174 + + + + + +-7734715135337881451 + +simple-shuffle-split + + + +-4202570559908674985->-7734715135337881451 + + + + + +-6753417271959464076 + +simple-shuffle-split + + + +-4202570559908674985->-6753417271959464076 + + + + + +-4518684953114053501 + +simple-shuffle-split + + + +-4202570559908674985->-4518684953114053501 + + + + + +-1450263677678646526 + +simple-shuffle-split + + + +-4202570559908674985->-1450263677678646526 + + + + + +-1667001532951134751 + +simple-shuffle-split + + + +-4202570559908674985->-1667001532951134751 + + + + + +-5307840728950325712 + +simple-shuffle-split + + + +-4202570559908674985->-5307840728950325712 + + + + + +-2505520067161621137 + +simple-shuffle-split + + + +-4202570559908674985->-2505520067161621137 + + + + + +-2718937416079421362 + +simple-shuffle-split + + + +-4202570559908674985->-2718937416079421362 + + + + + +-986208786096426387 + +simple-shuffle-split + + + +-4202570559908674985->-986208786096426387 + + + + + +-762951365652515412 + +simple-shuffle-split + + + +-4202570559908674985->-762951365652515412 + + + + + +783587729709451963 + +simple-shuffle-split + + + +-4202570559908674985->783587729709451963 + + + + + +1755994873805821338 + +simple-shuffle-split + + + +-4202570559908674985->1755994873805821338 + + + + + +6164554445352025113 + +simple-shuffle-split + + + +-4202570559908674985->6164554445352025113 + + + + + +7207928434975675688 + +simple-shuffle-split + + + +-4202570559908674985->7207928434975675688 + + + + + +8190334666499398663 + +simple-shuffle-split + + + +-4202570559908674985->8190334666499398663 + + + + + +283700088515866438 + +simple-shuffle-split + + + +-4202570559908674985->283700088515866438 + + + + + +-8416669703028930203 + +simple-shuffle-split + + + +-4202570559908674985->-8416669703028930203 + + + + + +-6759457844188343228 + +simple-shuffle-split + + + +-4202570559908674985->-6759457844188343228 + + + + + +-6976159420306875053 + +simple-shuffle-split + + + +-4202570559908674985->-6976159420306875053 + + + + + +4758632420524321938 + +simple-shuffle-split + + + +-4202570559908674985->4758632420524321938 + + + + + +-2751506485177666703 + +simple-shuffle-split + + + +-4202570559908674985->-2751506485177666703 + + + + + +-1314437711803948928 + +simple-shuffle-split + + + +-4202570559908674985->-1314437711803948928 + + + + + +-4202570559907592460 + +7 + + + +6569962593908858050->-4202570559907592460 + + + + + +-5754122772232681887 + +simple-shuffle-split + + + +-4202570559907592460->-5754122772232681887 + + + + + +-6797501304529601662 + +simple-shuffle-split + + + +-4202570559907592460->-6797501304529601662 + + + + + +-8329741654296963837 + +simple-shuffle-split + + + +-4202570559907592460->-8329741654296963837 + + + + + +300819336505097204 + +simple-shuffle-split + + + +-4202570559907592460->300819336505097204 + + + + + +7650154878988875029 + +simple-shuffle-split + + + +-4202570559907592460->7650154878988875029 + + + + + +8438954423858137654 + +simple-shuffle-split + + + +-4202570559907592460->8438954423858137654 + + + + + +7364439732854986679 + +simple-shuffle-split + + + +-4202570559907592460->7364439732854986679 + + + + + +-5258933739439424296 + +simple-shuffle-split + + + +-4202570559907592460->-5258933739439424296 + + + + + +3218574475053200329 + +simple-shuffle-split + + + +-4202570559907592460->3218574475053200329 + + + + + +534348247021558154 + +simple-shuffle-split + + + +-4202570559907592460->534348247021558154 + + + + + +9163767995818919595 + +simple-shuffle-split + + + +-4202570559907592460->9163767995818919595 + + + + + +-2556760286676292196 + +simple-shuffle-split + + + +-4202570559907592460->-2556760286676292196 + + + + + +-2410994982461369571 + +simple-shuffle-split + + + +-4202570559907592460->-2410994982461369571 + + + + + +-3317887246322628546 + +simple-shuffle-split + + + +-4202570559907592460->-3317887246322628546 + + + + + +3498586838952249695 + +simple-shuffle-split + + + +-4202570559907592460->3498586838952249695 + + + + + +-6477424979172241696 + +simple-shuffle-split + + + +-4202570559907592460->-6477424979172241696 + + + + + +-6854852325354253871 + +simple-shuffle-split + + + +-4202570559907592460->-6854852325354253871 + + + + + +-7898226314995224846 + +simple-shuffle-split + + + +-4202570559907592460->-7898226314995224846 + + + + + +6232101836192463795 + +simple-shuffle-split + + + +-4202570559907592460->6232101836192463795 + + + + + +5188723305021370020 + +simple-shuffle-split + + + +-4202570559907592460->5188723305021370020 + + + + + +3730873507950792645 + +simple-shuffle-split + + + +-4202570559907592460->3730873507950792645 + + + + + +3436658576584015270 + +simple-shuffle-split + + + +-4202570559907592460->3436658576584015270 + + + + + +1189617517990062695 + +simple-shuffle-split + + + +-4202570559907592460->1189617517990062695 + + + + + +1899615472660797320 + +simple-shuffle-split + + + +-4202570559907592460->1899615472660797320 + + + + + +1000458774366247545 + +simple-shuffle-split + + + +-4202570559907592460->1000458774366247545 + + + + + +-2197461496226689030 + +simple-shuffle-split + + + +-4202570559907592460->-2197461496226689030 + + + + + +-3241944988014194405 + +simple-shuffle-split + + + +-4202570559907592460->-3241944988014194405 + + + + + +-5926207496342939380 + +simple-shuffle-split + + + +-4202570559907592460->-5926207496342939380 + + + + + +-6220399989235439155 + +simple-shuffle-split + + + +-4202570559907592460->-6220399989235439155 + + + + + +-7827053363325109330 + +simple-shuffle-split + + + +-4202570559907592460->-7827053363325109330 + + + + + +-8874857355924774705 + +simple-shuffle-split + + + +-4202570559907592460->-8874857355924774705 + + + + + +8598339425830696336 + +simple-shuffle-split + + + +-4202570559907592460->8598339425830696336 + + + + + +6556737475064236161 + +simple-shuffle-split + + + +-4202570559907592460->6556737475064236161 + + + + + +5583190180858503586 + +simple-shuffle-split + + + +-4202570559907592460->5583190180858503586 + + + + + +5132444328813202211 + +simple-shuffle-split + + + +-4202570559907592460->5132444328813202211 + + + + + +2897713146186031636 + +simple-shuffle-split + + + +-4202570559907592460->2897713146186031636 + + + + + +-227312666293160139 + +simple-shuffle-split + + + +-4202570559907592460->-227312666293160139 + + + + + +-81564023991270314 + +simple-shuffle-split + + + +-4202570559907592460->-81564023991270314 + + + + + +6780143860165847511 + +simple-shuffle-split + + + +-4202570559907592460->6780143860165847511 + + + + + +-4218574893834009864 + +simple-shuffle-split + + + +-4202570559907592460->-4218574893834009864 + + + + + +-5190977504125154839 + +simple-shuffle-split + + + +-4202570559907592460->-5190977504125154839 + + + + + +-4560940270669059414 + +simple-shuffle-split + + + +-4202570559907592460->-4560940270669059414 + + + + + +-5533347414765428789 + +simple-shuffle-split + + + +-4202570559907592460->-5533347414765428789 + + + + + +-8732371510795760964 + +simple-shuffle-split + + + +-4202570559907592460->-8732371510795760964 + + + + + +8670998572129673277 + +simple-shuffle-split + + + +-4202570559907592460->8670998572129673277 + + + + + +-2864485405130455714 + +simple-shuffle-split + + + +-4202570559907592460->-2864485405130455714 + + + + + +5761341498825030527 + +simple-shuffle-split + + + +-4202570559907592460->5761341498825030527 + + + + + +4145887991457079552 + +simple-shuffle-split + + + +-4202570559907592460->4145887991457079552 + + + + + +3111368388735931377 + +simple-shuffle-split + + + +-4202570559907592460->3111368388735931377 + + + + + +-4202570559919500235 + +8 + + + +-5766413972891547541->-4202570559919500235 + + + + + +4857179192858259762 + +simple-shuffle-split + + + +-4202570559919500235->4857179192858259762 + + + + + +-820768228302314479 + +simple-shuffle-split + + + +-4202570559919500235->-820768228302314479 + + + + + +-7800744669336511004 + +simple-shuffle-split + + + +-4202570559919500235->-7800744669336511004 + + + + + +-5115682359529666829 + +simple-shuffle-split + + + +-4202570559919500235->-5115682359529666829 + + + + + +7668218353096620262 + +simple-shuffle-split + + + +-4202570559919500235->7668218353096620262 + + + + + +829811772141247237 + +simple-shuffle-split + + + +-4202570559919500235->829811772141247237 + + + + + +6832686957933287112 + +simple-shuffle-split + + + +-4202570559919500235->6832686957933287112 + + + + + +7886023711490110887 + +simple-shuffle-split + + + +-4202570559919500235->7886023711490110887 + + + + + +3686465368105103162 + +simple-shuffle-split + + + +-4202570559919500235->3686465368105103162 + + + + + +-4799897234346568263 + +simple-shuffle-split + + + +-4202570559919500235->-4799897234346568263 + + + + + +1171952605662011212 + +simple-shuffle-split + + + +-4202570559919500235->1171952605662011212 + + + + + +3406683789397687387 + +simple-shuffle-split + + + +-4202570559919500235->3406683789397687387 + + + + + +-1807360966765329938 + +simple-shuffle-split + + + +-4202570559919500235->-1807360966765329938 + + + + + +-1953109529185534963 + +simple-shuffle-split + + + +-4202570559919500235->-1953109529185534963 + + + + + +-5978792128170813488 + +simple-shuffle-split + + + +-4202570559919500235->-5978792128170813488 + + + + + +-4564177524699932913 + +simple-shuffle-split + + + +-4202570559919500235->-4564177524699932913 + + + + + +-71156936529995070 + +simple-shuffle-split + + + +-4202570559919500235->-71156936529995070 + + + + + +-8107156654259699295 + +simple-shuffle-split + + + +-4202570559919500235->-8107156654259699295 + + + + + +-4317989564525065420 + +simple-shuffle-split + + + +-4202570559919500235->-4317989564525065420 + + + + + +7364721822182498371 + +simple-shuffle-split + + + +-4202570559919500235->7364721822182498371 + + + + + +-5153537652708869770 + +simple-shuffle-split + + + +-4202570559919500235->-5153537652708869770 + + + + + +-4100178531726049195 + +simple-shuffle-split + + + +-4202570559919500235->-4100178531726049195 + + + + + +624997638727171096 + +simple-shuffle-split + + + +-4202570559919500235->624997638727171096 + + + + + +2850871046876046071 + +simple-shuffle-split + + + +-4202570559919500235->2850871046876046071 + + + + + +8743900272582271946 + +simple-shuffle-split + + + +-4202570559919500235->8743900272582271946 + + + + + +-8575075640969064695 + +simple-shuffle-split + + + +-4202570559919500235->-8575075640969064695 + + + + + +3388576755599659996 + +simple-shuffle-split + + + +-4202570559919500235->3388576755599659996 + + + + + +4433055698062862571 + +simple-shuffle-split + + + +-4202570559919500235->4433055698062862571 + + + + + +-7924404998424097154 + +simple-shuffle-split + + + +-4202570559919500235->-7924404998424097154 + + + + + +1821972620273232221 + +simple-shuffle-split + + + +-4202570559919500235->1821972620273232221 + + + + + +213751366659294496 + +simple-shuffle-split + + + +-4202570559919500235->213751366659294496 + + + + + +-644929106138540129 + +simple-shuffle-split + + + +-4202570559919500235->-644929106138540129 + + + + + +-2886874508824520878 + +simple-shuffle-split + + + +-4202570559919500235->-2886874508824520878 + + + + + +-1502706381124752847 + +simple-shuffle-split + + + +-4202570559919500235->-1502706381124752847 + + + + + +-3229241022986316028 + +simple-shuffle-split + + + +-4202570559919500235->-3229241022986316028 + + + + + +-3931357992817846253 + +simple-shuffle-split + + + +-4202570559919500235->-3931357992817846253 + + + + + +-7361806797601257978 + +simple-shuffle-split + + + +-4202570559919500235->-7361806797601257978 + + + + + +-4273723370796042203 + +simple-shuffle-split + + + +-4202570559919500235->-4273723370796042203 + + + + + +-1642911886437672728 + +simple-shuffle-split + + + +-4202570559919500235->-1642911886437672728 + + + + + +-8480676302685834553 + +simple-shuffle-split + + + +-4202570559919500235->-8480676302685834553 + + + + + +5516473937054446938 + +simple-shuffle-split + + + +-4202570559919500235->5516473937054446938 + + + + + +6986504411599076313 + +simple-shuffle-split + + + +-4202570559919500235->6986504411599076313 + + + + + +-5526168150762708628 + +simple-shuffle-split + + + +-4202570559919500235->-5526168150762708628 + + + + + +4441959315921789563 + +simple-shuffle-split + + + +-4202570559919500235->4441959315921789563 + + + + + +383377347104769038 + +simple-shuffle-split + + + +-4202570559919500235->383377347104769038 + + + + + +1426773748581242413 + +simple-shuffle-split + + + +-4202570559919500235->1426773748581242413 + + + + + +7255394766097493488 + +simple-shuffle-split + + + +-4202570559919500235->7255394766097493488 + + + + + +-1223217293854909137 + +simple-shuffle-split + + + +-4202570559919500235->-1223217293854909137 + + + + + +-6246766954595371038 + +simple-shuffle-split + + + +-4202570559919500235->-6246766954595371038 + + + + + +-4202570559918417710 + +9 + + + +-7871270668270266016->-4202570559918417710 + + + + + +-1919952402746357421 + +simple-shuffle-split + + + +-4202570559918417710->-1919952402746357421 + + + + + +-2973331147586494396 + +simple-shuffle-split + + + +-4202570559918417710->-2973331147586494396 + + + + + +1239552412960182129 + +simple-shuffle-split + + + +-4202570559918417710->1239552412960182129 + + + + + +8056059423068391570 + +simple-shuffle-split + + + +-4202570559918417710->8056059423068391570 + + + + + +-7510353515772233721 + +simple-shuffle-split + + + +-4202570559918417710->-7510353515772233721 + + + + + +-8492755242329198296 + +simple-shuffle-split + + + +-4202570559918417710->-8492755242329198296 + + + + + +-4438895229115951771 + +simple-shuffle-split + + + +-4202570559918417710->-4438895229115951771 + + + + + +4160472001651424070 + +simple-shuffle-split + + + +-4202570559918417710->4160472001651424070 + + + + + +6457050598155703995 + +simple-shuffle-split + + + +-4202570559918417710->6457050598155703995 + + + + + +4367645068351906220 + +simple-shuffle-split + + + +-4202570559918417710->4367645068351906220 + + + + + +6711742163229726745 + +simple-shuffle-split + + + +-4202570559918417710->6711742163229726745 + + + + + +5747053807415849370 + +simple-shuffle-split + + + +-4202570559918417710->5747053807415849370 + + + + + +1524425491201661295 + +simple-shuffle-split + + + +-4202570559918417710->1524425491201661295 + + + + + +-1663785940783932880 + +simple-shuffle-split + + + +-4202570559918417710->-1663785940783932880 + + + + + +-5457396988436939155 + +simple-shuffle-split + + + +-4202570559918417710->-5457396988436939155 + + + + + +1258833679108488270 + +simple-shuffle-split + + + +-4202570559918417710->1258833679108488270 + + + + + +1923945623603648163 + +simple-shuffle-split + + + +-4202570559918417710->1923945623603648163 + + + + + +-7751340781854472812 + +simple-shuffle-split + + + +-4202570559918417710->-7751340781854472812 + + + + + +-4986961968399474687 + +simple-shuffle-split + + + +-4202570559918417710->-4986961968399474687 + + + + + +3491654631957224738 + +simple-shuffle-split + + + +-4202570559918417710->3491654631957224738 + + + + + +4839411449728680279 + +simple-shuffle-split + + + +-4202570559918417710->4839411449728680279 + + + + + +-6239170802135122312 + +simple-shuffle-split + + + +-4202570559918417710->-6239170802135122312 + + + + + +-1050920043447597387 + +simple-shuffle-split + + + +-4202570559918417710->-1050920043447597387 + + + + + +-1957845225491153962 + +simple-shuffle-split + + + +-4202570559918417710->-1957845225491153962 + + + + + +1006443045261147979 + +simple-shuffle-split + + + +-4202570559918417710->1006443045261147979 + + + + + +7793764927223571004 + +simple-shuffle-split + + + +-4202570559918417710->7793764927223571004 + + + + + +-5451425430555060887 + +simple-shuffle-split + + + +-4202570559918417710->-5451425430555060887 + + + + + +-6432723295007343062 + +simple-shuffle-split + + + +-4202570559918417710->-6432723295007343062 + + + + + +2550528581993434879 + +simple-shuffle-split + + + +-4202570559918417710->2550528581993434879 + + + + + +2203096122514021504 + +simple-shuffle-split + + + +-4202570559918417710->2203096122514021504 + + + + + +7983857907155101629 + +simple-shuffle-split + + + +-4202570559918417710->7983857907155101629 + + + + + +-4725257084341792546 + +simple-shuffle-split + + + +-4202570559918417710->-4725257084341792546 + + + + + +4289347433630180147 + +simple-shuffle-split + + + +-4202570559918417710->4289347433630180147 + + + + + +4502727323944065572 + +simple-shuffle-split + + + +-4202570559918417710->4502727323944065572 + + + + + +-8797639653858374319 + +simple-shuffle-split + + + +-4202570559918417710->-8797639653858374319 + + + + + +-1286392425459809678 + +simple-shuffle-split + + + +-4202570559918417710->-1286392425459809678 + + + + + +-8512897700337289753 + +simple-shuffle-split + + + +-4202570559918417710->-8512897700337289753 + + + + + +27795034041502472 + +simple-shuffle-split + + + +-4202570559918417710->27795034041502472 + + + + + +-7146791884187708603 + +simple-shuffle-split + + + +-4202570559918417710->-7146791884187708603 + + + + + +9146138459287609638 + +simple-shuffle-split + + + +-4202570559918417710->9146138459287609638 + + + + + +-5262540898532318053 + +simple-shuffle-split + + + +-4202570559918417710->-5262540898532318053 + + + + + +1660625376724558988 + +simple-shuffle-split + + + +-4202570559918417710->1660625376724558988 + + + + + +7466962508328542713 + +simple-shuffle-split + + + +-4202570559918417710->7466962508328542713 + + + + + +4635759870517250938 + +simple-shuffle-split + + + +-4202570559918417710->4635759870517250938 + + + + + +7698963518950006863 + +simple-shuffle-split + + + +-4202570559918417710->7698963518950006863 + + + + + +-4004555745761116912 + +simple-shuffle-split + + + +-4202570559918417710->-4004555745761116912 + + + + + +678219146343982413 + +simple-shuffle-split + + + +-4202570559918417710->678219146343982413 + + + + + +8663651900224872238 + +simple-shuffle-split + + + +-4202570559918417710->8663651900224872238 + + + + + +-672948230417510269 + +simple-shuffle-split + + + +-4202570559918417710->-672948230417510269 + + + + + +-4202570559921665285 + +10 + + + +-3559868500111997791->-4202570559921665285 + + + + + +-9081145470470620508 + +simple-shuffle-split + + + +-4202570559921665285->-9081145470470620508 + + + + + +-8104277598983161933 + +simple-shuffle-split + + + +-4202570559921665285->-8104277598983161933 + + + + + +3631655476075875058 + +simple-shuffle-split + + + +-4202570559921665285->3631655476075875058 + + + + + +5011358343687835089 + +simple-shuffle-split + + + +-4202570559921665285->5011358343687835089 + + + + + +6119838159620707208 + +simple-shuffle-split + + + +-4202570559921665285->6119838159620707208 + + + + + +7172107410091169383 + +simple-shuffle-split + + + +-4202570559921665285->7172107410091169383 + + + + + +-883174416731428442 + +simple-shuffle-split + + + +-4202570559921665285->-883174416731428442 + + + + + +8963094060282452933 + +simple-shuffle-split + + + +-4202570559921665285->8963094060282452933 + + + + + +-1885763688648309492 + +simple-shuffle-split + + + +-4202570559921665285->-1885763688648309492 + + + + + +-195749352510591717 + +simple-shuffle-split + + + +-4202570559921665285->-195749352510591717 + + + + + +777797939478129658 + +simple-shuffle-split + + + +-4202570559921665285->777797939478129658 + + + + + +1812314092581811833 + +simple-shuffle-split + + + +-4202570559921665285->1812314092581811833 + + + + + +-6325228452822740592 + +simple-shuffle-split + + + +-4202570559921665285->-6325228452822740592 + + + + + +-3593880342056960817 + +simple-shuffle-split + + + +-4202570559921665285->-3593880342056960817 + + + + + +-3785077104280770642 + +simple-shuffle-split + + + +-4202570559921665285->-3785077104280770642 + + + + + +-2043525817259567667 + +simple-shuffle-split + + + +-4202570559921665285->-2043525817259567667 + + + + + +-3978427634718832652 + +simple-shuffle-split + + + +-4202570559921665285->-3978427634718832652 + + + + + +5842185448930341123 + +simple-shuffle-split + + + +-4202570559921665285->5842185448930341123 + + + + + +6274789343135258498 + +simple-shuffle-split + + + +-4202570559921665285->6274789343135258498 + + + + + +7247228270304191073 + +simple-shuffle-split + + + +-4202570559921665285->7247228270304191073 + + + + + +-2317476625053574952 + +simple-shuffle-split + + + +-4202570559921665285->-2317476625053574952 + + + + + +7641791913198298039 + +simple-shuffle-split + + + +-4202570559921665285->7641791913198298039 + + + + + +2950208301790303798 + +simple-shuffle-split + + + +-4202570559921665285->2950208301790303798 + + + + + +-5445140947235528427 + +simple-shuffle-split + + + +-4202570559921665285->-5445140947235528427 + + + + + +-6880333718694692452 + +simple-shuffle-split + + + +-4202570559921665285->-6880333718694692452 + + + + + +3789311093500666539 + +simple-shuffle-split + + + +-4202570559921665285->3789311093500666539 + + + + + +-3720827768969603702 + +simple-shuffle-split + + + +-4202570559921665285->-3720827768969603702 + + + + + +-3290987550281756727 + +simple-shuffle-split + + + +-4202570559921665285->-3290987550281756727 + + + + + +6046981720907082464 + +simple-shuffle-split + + + +-4202570559921665285->6046981720907082464 + + + + + +8755884661202425183 + +simple-shuffle-split + + + +-4202570559921665285->8755884661202425183 + + + + + +9026241064131966014 + +simple-shuffle-split + + + +-4202570559921665285->9026241064131966014 + + + + + +-6615850267021926627 + +simple-shuffle-split + + + +-4202570559921665285->-6615850267021926627 + + + + + +2677271561227039940 + +simple-shuffle-split + + + +-4202570559921665285->2677271561227039940 + + + + + +4172089594374146515 + +simple-shuffle-split + + + +-4202570559921665285->4172089594374146515 + + + + + +-3189365622618828526 + +simple-shuffle-split + + + +-4202570559921665285->-3189365622618828526 + + + + + +8392345446943969265 + +simple-shuffle-split + + + +-4202570559921665285->8392345446943969265 + + + + + +-2290822178834908760 + +simple-shuffle-split + + + +-4202570559921665285->-2290822178834908760 + + + + + +-2498664809858590585 + +simple-shuffle-split + + + +-4202570559921665285->-2498664809858590585 + + + + + +228119519058664390 + +simple-shuffle-split + + + +-4202570559921665285->228119519058664390 + + + + + +-7290855904366273051 + +simple-shuffle-split + + + +-4202570559921665285->-7290855904366273051 + + + + + +-6871161276311330260 + +simple-shuffle-split + + + +-4202570559921665285->-6871161276311330260 + + + + + +-5826678890829734085 + +simple-shuffle-split + + + +-4202570559921665285->-5826678890829734085 + + + + + +-5398117299784551910 + +simple-shuffle-split + + + +-4202570559921665285->-5398117299784551910 + + + + + +5490707045731776665 + +simple-shuffle-split + + + +-4202570559921665285->5490707045731776665 + + + + + +-5284564422600705360 + +simple-shuffle-split + + + +-4202570559921665285->-5284564422600705360 + + + + + +6442472264278645231 + +simple-shuffle-split + + + +-4202570559921665285->6442472264278645231 + + + + + +6290162162280269006 + +simple-shuffle-split + + + +-4202570559921665285->6290162162280269006 + + + + + +7260271865405100781 + +simple-shuffle-split + + + +-4202570559921665285->7260271865405100781 + + + + + +6688660259925691924 + +simple-shuffle-split + + + +-4202570559921665285->6688660259925691924 + + + + + +-4202570559920582760 + +11 + + + +-4660231804639902666->-4202570559920582760 + + + + + +-6890795871647074779 + +simple-shuffle-split + + + +-4202570559920582760->-6890795871647074779 + + + + + +-7140679721968040954 + +simple-shuffle-split + + + +-4202570559920582760->-7140679721968040954 + + + + + +8996942618686527687 + +simple-shuffle-split + + + +-4202570559920582760->8996942618686527687 + + + + + +-8599211999760001304 + +simple-shuffle-split + + + +-4202570559920582760->-8599211999760001304 + + + + + +1057864508383055921 + +simple-shuffle-split + + + +-4202570559920582760->1057864508383055921 + + + + + +14490516525073746 + +simple-shuffle-split + + + +-4202570559920582760->14490516525073746 + + + + + +-4301887946267048429 + +simple-shuffle-split + + + +-4202570559920582760->-4301887946267048429 + + + + + +-5349728331088241404 + +simple-shuffle-split + + + +-4202570559920582760->-5349728331088241404 + + + + + +1538001814289544237 + +simple-shuffle-split + + + +-4202570559920582760->1538001814289544237 + + + + + +-1192222882688343538 + +simple-shuffle-split + + + +-4202570559920582760->-1192222882688343538 + + + + + +-1002129980317564113 + +simple-shuffle-split + + + +-4202570559920582760->-1002129980317564113 + + + + + +-2045508512579843088 + +simple-shuffle-split + + + +-4202570559920582760->-2045508512579843088 + + + + + +8803048626519589337 + +simple-shuffle-split + + + +-4202570559920582760->8803048626519589337 + + + + + +5566206497322902362 + +simple-shuffle-split + + + +-4202570559920582760->5566206497322902362 + + + + + +4521722977822756987 + +simple-shuffle-split + + + +-4202570559920582760->4521722977822756987 + + + + + +1842194514200053612 + +simple-shuffle-split + + + +-4202570559920582760->1842194514200053612 + + + + + +-1799284193455936395 + +simple-shuffle-split + + + +-4202570559920582760->-1799284193455936395 + + + + + +-2138656212072600170 + +simple-shuffle-split + + + +-4202570559920582760->-2138656212072600170 + + + + + +-3183139705003251945 + +simple-shuffle-split + + + +-4202570559920582760->-3183139705003251945 + + + + + +2504770552126039096 + +simple-shuffle-split + + + +-4202570559920582760->2504770552126039096 + + + + + +2084240930883034305 + +simple-shuffle-split + + + +-4202570559920582760->2084240930883034305 + + + + + +1039757474567708130 + +simple-shuffle-split + + + +-4202570559920582760->1039757474567708130 + + + + + +-526846616418495645 + +simple-shuffle-split + + + +-4202570559920582760->-526846616418495645 + + + + + +8173541034346624596 + +simple-shuffle-split + + + +-4202570559920582760->8173541034346624596 + + + + + +5532796804919332221 + +simple-shuffle-split + + + +-4202570559920582760->5532796804919332221 + + + + + +4489404900719359646 + +simple-shuffle-split + + + +-4202570559920582760->4489404900719359646 + + + + + +-5962897762206347329 + +simple-shuffle-split + + + +-4202570559920582760->-5962897762206347329 + + + + + +1325470021017036096 + +simple-shuffle-split + + + +-4202570559920582760->1325470021017036096 + + + + + +-7464465281595259095 + +simple-shuffle-split + + + +-4202570559920582760->-7464465281595259095 + + + + + +-8958206731869314070 + +simple-shuffle-split + + + +-4202570559920582760->-8958206731869314070 + + + + + +-1519002932979587829 + +simple-shuffle-split + + + +-4202570559920582760->-1519002932979587829 + + + + + +5268313272325657596 + +simple-shuffle-split + + + +-4202570559920582760->5268313272325657596 + + + + + +3203007113379248389 + +simple-shuffle-split + + + +-4202570559920582760->3203007113379248389 + + + + + +9038576741257715430 + +simple-shuffle-split + + + +-4202570559920582760->9038576741257715430 + + + + + +7995198210103942055 + +simple-shuffle-split + + + +-4202570559920582760->7995198210103942055 + + + + + +-2424368759682394936 + +simple-shuffle-split + + + +-4202570559920582760->-2424368759682394936 + + + + + +7123921663747574289 + +simple-shuffle-split + + + +-4202570559920582760->7123921663747574289 + + + + + +-3812683538623662286 + +simple-shuffle-split + + + +-4202570559920582760->-3812683538623662286 + + + + + +-4865185373210338061 + +simple-shuffle-split + + + +-4202570559920582760->-4865185373210338061 + + + + + +-5196785536003959836 + +simple-shuffle-split + + + +-4202570559920582760->-5196785536003959836 + + + + + +497639599935611405 + +simple-shuffle-split + + + +-4202570559920582760->497639599935611405 + + + + + +-1737109480709578770 + +simple-shuffle-split + + + +-4202570559920582760->-1737109480709578770 + + + + + +7518420251712912655 + +simple-shuffle-split + + + +-4202570559920582760->7518420251712912655 + + + + + +6399550792077065680 + +simple-shuffle-split + + + +-4202570559920582760->6399550792077065680 + + + + + +5015905947640216505 + +simple-shuffle-split + + + +-4202570559920582760->5015905947640216505 + + + + + +-6666808845557017286 + +simple-shuffle-split + + + +-4202570559920582760->-6666808845557017286 + + + + + +-6448962624356234661 + +simple-shuffle-split + + + +-4202570559920582760->-6448962624356234661 + + + + + +-7502340241846176436 + +simple-shuffle-split + + + +-4202570559920582760->-7502340241846176436 + + + + + +-3991005168221952427 + +simple-shuffle-split + + + +-4202570559920582760->-3991005168221952427 + + + + + +-4202570559915170135 + +12 + + + +8401877563929125775->-4202570559915170135 + + + + + +-8195566856495047834 + +simple-shuffle-split + + + +-4202570559915170135->-8195566856495047834 + + + + + +-5127127744061390459 + +simple-shuffle-split + + + +-4202570559915170135->-5127127744061390459 + + + + + +7657840427328354632 + +simple-shuffle-split + + + +-4202570559915170135->7657840427328354632 + + + + + +8087417423913709607 + +simple-shuffle-split + + + +-4202570559915170135->8087417423913709607 + + + + + +-970229545114285134 + +simple-shuffle-split + + + +-4202570559915170135->-970229545114285134 + + + + + +-71109137058747759 + +simple-shuffle-split + + + +-4202570559915170135->-71109137058747759 + + + + + +-5272894220001373084 + +simple-shuffle-split + + + +-4202570559915170135->-5272894220001373084 + + + + + +6460919666168667507 + +simple-shuffle-split + + + +-4202570559915170135->6460919666168667507 + + + + + +720494030768284782 + +simple-shuffle-split + + + +-4202570559915170135->720494030768284782 + + + + + +1014704386223945357 + +simple-shuffle-split + + + +-4202570559915170135->1014704386223945357 + + + + + +-3029156468617589168 + +simple-shuffle-split + + + +-4202570559915170135->-3029156468617589168 + + + + + +-2056712994393326193 + +simple-shuffle-split + + + +-4202570559915170135->-2056712994393326193 + + + + + +4586014001545610682 + +simple-shuffle-split + + + +-4202570559915170135->4586014001545610682 + + + + + +6684329456651443257 + +simple-shuffle-split + + + +-4202570559915170135->6684329456651443257 + + + + + +2564698968433181132 + +simple-shuffle-split + + + +-4202570559915170135->2564698968433181132 + + + + + +3612503031959884507 + +simple-shuffle-split + + + +-4202570559915170135->3612503031959884507 + + + + + +-4329487973330979850 + +simple-shuffle-split + + + +-4202570559915170135->-4329487973330979850 + + + + + +-3898789289905535275 + +simple-shuffle-split + + + +-4202570559915170135->-3898789289905535275 + + + + + +1449080235178853016 + +simple-shuffle-split + + + +-4202570559915170135->1449080235178853016 + + + + + +-5301895151825663625 + +simple-shuffle-split + + + +-4202570559915170135->-5301895151825663625 + + + + + +1265336373009891650 + +simple-shuffle-split + + + +-4202570559915170135->1265336373009891650 + + + + + +-5525301545795319775 + +simple-shuffle-split + + + +-4202570559915170135->-5525301545795319775 + + + + + +7193080811465441716 + +simple-shuffle-split + + + +-4202570559915170135->7193080811465441716 + + + + + +1456537633514785475 + +simple-shuffle-split + + + +-4202570559915170135->1456537633514785475 + + + + + +3430295500231488766 + +simple-shuffle-split + + + +-4202570559915170135->3430295500231488766 + + + + + +4403823161694539741 + +simple-shuffle-split + + + +-4202570559915170135->4403823161694539741 + + + + + +-794393757369996384 + +simple-shuffle-split + + + +-4202570559915170135->-794393757369996384 + + + + + +1369402351752040991 + +simple-shuffle-split + + + +-4202570559915170135->1369402351752040991 + + + + + +8945294019369544266 + +simple-shuffle-split + + + +-4202570559915170135->8945294019369544266 + + + + + +-8448112129127405175 + +simple-shuffle-split + + + +-4202570559915170135->-8448112129127405175 + + + + + +-2367666422473586084 + +simple-shuffle-split + + + +-4202570559915170135->-2367666422473586084 + + + + + +7479689109361827691 + +simple-shuffle-split + + + +-4202570559915170135->7479689109361827691 + + + + + +-5637290844825921402 + +simple-shuffle-split + + + +-4202570559915170135->-5637290844825921402 + + + + + +5441276432305012389 + +simple-shuffle-split + + + +-4202570559915170135->5441276432305012389 + + + + + +1443006734791125864 + +simple-shuffle-split + + + +-4202570559915170135->1443006734791125864 + + + + + +2992160687078644039 + +simple-shuffle-split + + + +-4202570559915170135->2992160687078644039 + + + + + +32314639450770898 + +simple-shuffle-split + + + +-4202570559915170135->32314639450770898 + + + + + +-8442841942470446927 + +simple-shuffle-split + + + +-4202570559915170135->-8442841942470446927 + + + + + +4802345080923340164 + +simple-shuffle-split + + + +-4202570559915170135->4802345080923340164 + + + + + +5779212872598395539 + +simple-shuffle-split + + + +-4202570559915170135->5779212872598395539 + + + + + +-8348330662068540786 + +simple-shuffle-split + + + +-4202570559915170135->-8348330662068540786 + + + + + +2017284929772681389 + +simple-shuffle-split + + + +-4202570559915170135->2017284929772681389 + + + + + +-9166144148232219536 + +simple-shuffle-split + + + +-4202570559915170135->-9166144148232219536 + + + + + +9134833457262247855 + +simple-shuffle-split + + + +-4202570559915170135->9134833457262247855 + + + + + +-3934504491752526886 + +simple-shuffle-split + + + +-4202570559915170135->-3934504491752526886 + + + + + +399628203620032089 + +simple-shuffle-split + + + +-4202570559915170135->399628203620032089 + + + + + +4188831590262032364 + +simple-shuffle-split + + + +-4202570559915170135->4188831590262032364 + + + + + +-4911373419663142661 + +simple-shuffle-split + + + +-4202570559915170135->-4911373419663142661 + + + + + +6713277375217317398 + +simple-shuffle-split + + + +-4202570559915170135->6713277375217317398 + + + + + +-4202570559914087610 + +13 + + + +5748732521581009300->-4202570559914087610 + + + + + +-6292182916893524345 + +simple-shuffle-split + + + +-4202570559914087610->-6292182916893524345 + + + + + +-8977249768265132120 + +simple-shuffle-split + + + +-4202570559914087610->-8977249768265132120 + + + + + +-3240011315206611995 + +simple-shuffle-split + + + +-4202570559914087610->-3240011315206611995 + + + + + +-5997990422892518970 + +simple-shuffle-split + + + +-4202570559914087610->-5997990422892518970 + + + + + +-1711822607036620845 + +simple-shuffle-split + + + +-4202570559914087610->-1711822607036620845 + + + + + +-3453392277428092220 + +simple-shuffle-split + + + +-4202570559914087610->-3453392277428092220 + + + + + +132329273046955505 + +simple-shuffle-split + + + +-4202570559914087610->132329273046955505 + + + + + +-363847941606469870 + +simple-shuffle-split + + + +-4202570559914087610->-363847941606469870 + + + + + +1039934370594930671 + +simple-shuffle-split + + + +-4202570559914087610->1039934370594930671 + + + + + +-4549122266439504 + +simple-shuffle-split + + + +-4202570559914087610->-4549122266439504 + + + + + +2830884659620163821 + +simple-shuffle-split + + + +-4202570559914087610->2830884659620163821 + + + + + +2606502101454308046 + +simple-shuffle-split + + + +-4202570559914087610->2606502101454308046 + + + + + +4293542852048915771 + +simple-shuffle-split + + + +-4202570559914087610->4293542852048915771 + + + + + +3236746071205885996 + +simple-shuffle-split + + + +-4202570559914087610->3236746071205885996 + + + + + +8426119171842989721 + +simple-shuffle-split + + + +-4202570559914087610->8426119171842989721 + + + + + +7020363470148638746 + +simple-shuffle-split + + + +-4202570559914087610->7020363470148638746 + + + + + +-3560105438851128361 + +simple-shuffle-split + + + +-4202570559914087610->-3560105438851128361 + + + + + +3958892326573458680 + +simple-shuffle-split + + + +-4202570559914087610->3958892326573458680 + + + + + +9093106721763076405 + +simple-shuffle-split + + + +-4202570559914087610->9093106721763076405 + + + + + +-2516726907628073386 + +simple-shuffle-split + + + +-4202570559914087610->-2516726907628073386 + + + + + +2082470217162523939 + +simple-shuffle-split + + + +-4202570559914087610->2082470217162523939 + + + + + +467012126175878164 + +simple-shuffle-split + + + +-4202570559914087610->467012126175878164 + + + + + +-4773293742834941311 + +simple-shuffle-split + + + +-4202570559914087610->-4773293742834941311 + + + + + +3007127152366814114 + +simple-shuffle-split + + + +-4202570559914087610->3007127152366814114 + + + + + +2140391653956662655 + +simple-shuffle-split + + + +-4202570559914087610->2140391653956662655 + + + + + +1095912667101274880 + +simple-shuffle-split + + + +-4202570559914087610->1095912667101274880 + + + + + +7500470524465785405 + +simple-shuffle-split + + + +-4202570559914087610->7500470524465785405 + + + + + +6455970341909460830 + +simple-shuffle-split + + + +-4202570559914087610->6455970341909460830 + + + + + +-9150873633437163061 + +simple-shuffle-split + + + +-4202570559914087610->-9150873633437163061 + + + + + +-477095632995908420 + +simple-shuffle-split + + + +-4202570559914087610->-477095632995908420 + + + + + +-7767035994394917911 + +simple-shuffle-split + + + +-4202570559914087610->-7767035994394917911 + + + + + +-8749442215907449686 + +simple-shuffle-split + + + +-4202570559914087610->-8749442215907449686 + + + + + +-1307626031239725977 + +simple-shuffle-split + + + +-4202570559914087610->-1307626031239725977 + + + + + +-2288922725144735352 + +simple-shuffle-split + + + +-4202570559914087610->-2288922725144735352 + + + + + +2912615871964907973 + +simple-shuffle-split + + + +-4202570559914087610->2912615871964907973 + + + + + +-7952941697042880602 + +simple-shuffle-split + + + +-4202570559914087610->-7952941697042880602 + + + + + +-2824261163539161677 + +simple-shuffle-split + + + +-4202570559914087610->-2824261163539161677 + + + + + +4022629880567517348 + +simple-shuffle-split + + + +-4202570559914087610->4022629880567517348 + + + + + +8541916503811896273 + +simple-shuffle-split + + + +-4202570559914087610->8541916503811896273 + + + + + +7001925547217299698 + +simple-shuffle-split + + + +-4202570559914087610->7001925547217299698 + + + + + +6592883916739257039 + +simple-shuffle-split + + + +-4202570559914087610->6592883916739257039 + + + + + +6171062038668210064 + +simple-shuffle-split + + + +-4202570559914087610->6171062038668210064 + + + + + +269190575367459789 + +simple-shuffle-split + + + +-4202570559914087610->269190575367459789 + + + + + +7482041614087210414 + +simple-shuffle-split + + + +-4202570559914087610->7482041614087210414 + + + + + +-5672677791096911077 + +simple-shuffle-split + + + +-4202570559914087610->-5672677791096911077 + + + + + +-5609320180705912052 + +simple-shuffle-split + + + +-4202570559914087610->-5609320180705912052 + + + + + +-1501214846549582727 + +simple-shuffle-split + + + +-4202570559914087610->-1501214846549582727 + + + + + +5909033236342673914 + +simple-shuffle-split + + + +-4202570559914087610->5909033236342673914 + + + + + +4884914100384391607 + +simple-shuffle-split + + + +-4202570559914087610->4884914100384391607 + + + + + +-4202570559917335185 + +14 + + + +-7781630427718845691->-4202570559917335185 + + + + + +8927323017630980616 + +simple-shuffle-split + + + +-4202570559917335185->8927323017630980616 + + + + + +-8391689175056992025 + +simple-shuffle-split + + + +-4202570559917335185->-8391689175056992025 + + + + + +-6355486169794495450 + +simple-shuffle-split + + + +-4202570559917335185->-6355486169794495450 + + + + + +-5429702935223673275 + +simple-shuffle-split + + + +-4202570559917335185->-5429702935223673275 + + + + + +-5497757209277550300 + +simple-shuffle-split + + + +-4202570559917335185->-5497757209277550300 + + + + + +-2840795081896510925 + +simple-shuffle-split + + + +-4202570559917335185->-2840795081896510925 + + + + + +-1347494824610795150 + +simple-shuffle-split + + + +-4202570559917335185->-1347494824610795150 + + + + + +138496045440159825 + +simple-shuffle-split + + + +-4202570559917335185->138496045440159825 + + + + + +-1062854249564620784 + +simple-shuffle-split + + + +-4202570559917335185->-1062854249564620784 + + + + + +-1772753583852281009 + +simple-shuffle-split + + + +-4202570559917335185->-1772753583852281009 + + + + + +342152175067077166 + +simple-shuffle-split + + + +-4202570559917335185->342152175067077166 + + + + + +-8287282549571658675 + +simple-shuffle-split + + + +-4202570559917335185->-8287282549571658675 + + + + + +2256285881579871116 + +simple-shuffle-split + + + +-4202570559917335185->2256285881579871116 + + + + + +3309927778721288091 + +simple-shuffle-split + + + +-4202570559917335185->3309927778721288091 + + + + + +7616343511995871866 + +simple-shuffle-split + + + +-4202570559917335185->7616343511995871866 + + + + + +8651971472937736441 + +simple-shuffle-split + + + +-4202570559917335185->8651971472937736441 + + + + + +-4369877692225896616 + +simple-shuffle-split + + + +-4202570559917335185->-4369877692225896616 + + + + + +-3335361538048349641 + +simple-shuffle-split + + + +-4202570559917335185->-3335361538048349641 + + + + + +-3058901591902670666 + +simple-shuffle-split + + + +-4202570559917335185->-3058901591902670666 + + + + + +8103926044674373525 + +simple-shuffle-split + + + +-4202570559917335185->8103926044674373525 + + + + + +109516416963092084 + +simple-shuffle-split + + + +-4202570559917335185->109516416963092084 + + + + + +1152890416563293059 + +simple-shuffle-split + + + +-4202570559917335185->1152890416563293059 + + + + + +888375099788549634 + +simple-shuffle-split + + + +-4202570559917335185->888375099788549634 + + + + + +-5903375708756309791 + +simple-shuffle-split + + + +-4202570559917335185->-5903375708756309791 + + + + + +1172979360121986400 + +simple-shuffle-split + + + +-4202570559917335185->1172979360121986400 + + + + + +4419821694119671775 + +simple-shuffle-split + + + +-4202570559917335185->4419821694119671775 + + + + + +5472054629946677950 + +simple-shuffle-split + + + +-4202570559917335185->5472054629946677950 + + + + + +5803640844431655325 + +simple-shuffle-split + + + +-4202570559917335185->5803640844431655325 + + + + + +6925871783075459100 + +simple-shuffle-split + + + +-4202570559917335185->6925871783075459100 + + + + + +7176846129019340075 + +simple-shuffle-split + + + +-4202570559917335185->7176846129019340075 + + + + + +8716805301432867850 + +simple-shuffle-split + + + +-4202570559917335185->8716805301432867850 + + + + + +-8677401757047780791 + +simple-shuffle-split + + + +-4202570559917335185->-8677401757047780791 + + + + + +1214822034214673448 + +simple-shuffle-split + + + +-4202570559917335185->1214822034214673448 + + + + + +2763671927018754823 + +simple-shuffle-split + + + +-4202570559917335185->2763671927018754823 + + + + + +-5865744373252325818 + +simple-shuffle-split + + + +-4202570559917335185->-5865744373252325818 + + + + + +5650881614803919973 + +simple-shuffle-split + + + +-4202570559917335185->5650881614803919973 + + + + + +6769754488671655748 + +simple-shuffle-split + + + +-4202570559917335185->6769754488671655748 + + + + + +946347632455041107 + +simple-shuffle-split + + + +-4202570559917335185->946347632455041107 + + + + + +-7522301705704721518 + +simple-shuffle-split + + + +-4202570559917335185->-7522301705704721518 + + + + + +-8233273046902355343 + +simple-shuffle-split + + + +-4202570559917335185->-8233273046902355343 + + + + + +-21119354158739152 + +simple-shuffle-split + + + +-4202570559917335185->-21119354158739152 + + + + + +-167994189410623377 + +simple-shuffle-split + + + +-4202570559917335185->-167994189410623377 + + + + + +2940422150941566798 + +simple-shuffle-split + + + +-4202570559917335185->2940422150941566798 + + + + + +5174067341804075373 + +simple-shuffle-split + + + +-4202570559917335185->5174067341804075373 + + + + + +-3913823280536525652 + +simple-shuffle-split + + + +-4202570559917335185->-3913823280536525652 + + + + + +-2865982884630276677 + +simple-shuffle-split + + + +-4202570559917335185->-2865982884630276677 + + + + + +8618924021180934298 + +simple-shuffle-split + + + +-4202570559917335185->8618924021180934298 + + + + + +-8754413790638579943 + +simple-shuffle-split + + + +-4202570559917335185->-8754413790638579943 + + + + + +-3368405542318094984 + +simple-shuffle-split + + + +-4202570559917335185->-3368405542318094984 + + + + + +-4202570559916252660 + +15 + + + +-8944203705052616966->-4202570559916252660 + + + + + +3343389683881411673 + +simple-shuffle-split + + + +-4202570559916252660->3343389683881411673 + + + + + +2378696759915956698 + +simple-shuffle-split + + + +-4202570559916252660->2378696759915956698 + + + + + +894255442130601723 + +simple-shuffle-split + + + +-4202570559916252660->894255442130601723 + + + + + +-1845951633683128852 + +simple-shuffle-split + + + +-4202570559916252660->-1845951633683128852 + + + + + +-1703506801389939027 + +simple-shuffle-split + + + +-4202570559916252660->-1703506801389939027 + + + + + +-4820836271751174002 + +simple-shuffle-split + + + +-4202570559916252660->-4820836271751174002 + + + + + +-6935742065034205777 + +simple-shuffle-split + + + +-4202570559916252660->-6935742065034205777 + + + + + +-7431922684982193552 + +simple-shuffle-split + + + +-4202570559916252660->-7431922684982193552 + + + + + +-4951205630973585743 + +simple-shuffle-split + + + +-4202570559916252660->-4951205630973585743 + + + + + +-5994584163270505518 + +simple-shuffle-split + + + +-4202570559916252660->-5994584163270505518 + + + + + +-7544542421968240493 + +simple-shuffle-split + + + +-4202570559916252660->-7544542421968240493 + + + + + +-7838771230630022268 + +simple-shuffle-split + + + +-4202570559916252660->-7838771230630022268 + + + + + +7830765746973559973 + +simple-shuffle-split + + + +-4202570559916252660->7830765746973559973 + + + + + +6853880041761406598 + +simple-shuffle-split + + + +-4202570559916252660->6853880041761406598 + + + + + +3684918935761502023 + +simple-shuffle-split + + + +-4202570559916252660->3684918935761502023 + + + + + +2628962829197734248 + +simple-shuffle-split + + + +-4202570559916252660->2628962829197734248 + + + + + +-2093930094953632343 + +simple-shuffle-split + + + +-4202570559916252660->-2093930094953632343 + + + + + +4614566722004491882 + +simple-shuffle-split + + + +-4202570559916252660->4614566722004491882 + + + + + +3570084337614080907 + +simple-shuffle-split + + + +-4202570559916252660->3570084337614080907 + + + + + +1691948279327697532 + +simple-shuffle-split + + + +-4202570559916252660->1691948279327697532 + + + + + +-471847837553879043 + +simple-shuffle-split + + + +-4202570559916252660->-471847837553879043 + + + + + +-971366931658466018 + +simple-shuffle-split + + + +-4202570559916252660->-971366931658466018 + + + + + +-1879378071189612993 + +simple-shuffle-split + + + +-4202570559916252660->-1879378071189612993 + + + + + +-3986553634195349568 + +simple-shuffle-split + + + +-4202570559916252660->-3986553634195349568 + + + + + +9136614920439784257 + +simple-shuffle-split + + + +-4202570559916252660->9136614920439784257 + + + + + +7546500573262659682 + +simple-shuffle-split + + + +-4202570559916252660->7546500573262659682 + + + + + +4874965371404084707 + +simple-shuffle-split + + + +-4202570559916252660->4874965371404084707 + + + + + +5092775270219192532 + +simple-shuffle-split + + + +-4202570559916252660->5092775270219192532 + + + + + +-7190127266962414859 + +simple-shuffle-split + + + +-4202570559916252660->-7190127266962414859 + + + + + +9089482247924904982 + +simple-shuffle-split + + + +-4202570559916252660->9089482247924904982 + + + + + +-8569553867429052009 + +simple-shuffle-split + + + +-4202570559916252660->-8569553867429052009 + + + + + +8913643126852315832 + +simple-shuffle-split + + + +-4202570559916252660->8913643126852315832 + + + + + +1878888185578832441 + +simple-shuffle-split + + + +-4202570559916252660->1878888185578832441 + + + + + +8851114979586327482 + +simple-shuffle-split + + + +-4202570559916252660->8851114979586327482 + + + + + +-1047185744014511909 + +simple-shuffle-split + + + +-4202570559916252660->-1047185744014511909 + + + + + +-2666971879831775284 + +simple-shuffle-split + + + +-4202570559916252660->-2666971879831775284 + + + + + +-3640519129662643059 + +simple-shuffle-split + + + +-4202570559916252660->-3640519129662643059 + + + + + +4288471382651535982 + +simple-shuffle-split + + + +-4202570559916252660->4288471382651535982 + + + + + +-6791182912737565809 + +simple-shuffle-split + + + +-4202570559916252660->-6791182912737565809 + + + + + +-7834561444999844784 + +simple-shuffle-split + + + +-4202570559916252660->-7834561444999844784 + + + + + +-7111259237021060975 + +simple-shuffle-split + + + +-4202570559916252660->-7111259237021060975 + + + + + +-6820167154814940750 + +simple-shuffle-split + + + +-4202570559916252660->-6820167154814940750 + + + + + +-30961657404013709 + +simple-shuffle-split + + + +-4202570559916252660->-30961657404013709 + + + + + +7328342324561609316 + +simple-shuffle-split + + + +-4202570559916252660->7328342324561609316 + + + + + +8124945462860427141 + +simple-shuffle-split + + + +-4202570559916252660->8124945462860427141 + + + + + +7148059801988497766 + +simple-shuffle-split + + + +-4202570559916252660->7148059801988497766 + + + + + +-3859516798525198809 + +simple-shuffle-split + + + +-4202570559916252660->-3859516798525198809 + + + + + +2923370542239283016 + +simple-shuffle-split + + + +-4202570559916252660->2923370542239283016 + + + + + +-4183048609707923575 + +simple-shuffle-split + + + +-4202570559916252660->-4183048609707923575 + + + + + +-4202570559893519635 + +16 + + + +608660796933665091->-4202570559893519635 + + + + + +4059132380626355914 + +simple-shuffle-split + + + +-4202570559893519635->4059132380626355914 + + + + + +4405423563685434889 + +simple-shuffle-split + + + +-4202570559893519635->4405423563685434889 + + + + + +895202032546886364 + +simple-shuffle-split + + + +-4202570559893519635->895202032546886364 + + + + + +-5922413335747398677 + +simple-shuffle-split + + + +-4202570559893519635->-5922413335747398677 + + + + + +-1570576633620998786 + +simple-shuffle-split + + + +-4202570559893519635->-1570576633620998786 + + + + + +-30567807916130211 + +simple-shuffle-split + + + +-4202570559893519635->-30567807916130211 + + + + + +-2851604285814765536 + +simple-shuffle-split + + + +-4202570559893519635->-2851604285814765536 + + + + + +6898072677899988639 + +simple-shuffle-split + + + +-4202570559893519635->6898072677899988639 + + + + + +-4326762512391319102 + +simple-shuffle-split + + + +-4202570559893519635->-4326762512391319102 + + + + + +-3419855200566540127 + +simple-shuffle-split + + + +-4202570559893519635->-3419855200566540127 + + + + + +-4686850270006895052 + +simple-shuffle-split + + + +-4202570559893519635->-4686850270006895052 + + + + + +-3687834559722457277 + +simple-shuffle-split + + + +-4202570559893519635->-3687834559722457277 + + + + + +-8911552886174725002 + +simple-shuffle-split + + + +-4202570559893519635->-8911552886174725002 + + + + + +-7859302051307795627 + +simple-shuffle-split + + + +-4202570559893519635->-7859302051307795627 + + + + + +6560360780886250264 + +simple-shuffle-split + + + +-4202570559893519635->6560360780886250264 + + + + + +-8694851353339872777 + +simple-shuffle-split + + + +-4202570559893519635->-8694851353339872777 + + + + + +99806127217398330 + +simple-shuffle-split + + + +-4202570559893519635->99806127217398330 + + + + + +3286944390613790905 + +simple-shuffle-split + + + +-4202570559893519635->3286944390613790905 + + + + + +7120603515298850380 + +simple-shuffle-split + + + +-4202570559893519635->7120603515298850380 + + + + + +-2055094756512686245 + +simple-shuffle-split + + + +-4202570559893519635->-2055094756512686245 + + + + + +-5565374986417553170 + +simple-shuffle-split + + + +-4202570559893519635->-5565374986417553170 + + + + + +4947953008258536205 + +simple-shuffle-split + + + +-4202570559893519635->4947953008258536205 + + + + + +1412075056170588880 + +simple-shuffle-split + + + +-4202570559893519635->1412075056170588880 + + + + + +-6491105168949583345 + +simple-shuffle-split + + + +-4202570559893519635->-6491105168949583345 + + + + + +2959662237077566514 + +simple-shuffle-split + + + +-4202570559893519635->2959662237077566514 + + + + + +-5598784686580662511 + +simple-shuffle-split + + + +-4202570559893519635->-5598784686580662511 + + + + + +7585068148036350180 + +simple-shuffle-split + + + +-4202570559893519635->7585068148036350180 + + + + + +8566366047978131955 + +simple-shuffle-split + + + +-4202570559893519635->8566366047978131955 + + + + + +-2815644721923112986 + +simple-shuffle-split + + + +-4202570559893519635->-2815644721923112986 + + + + + +8372780637997478405 + +simple-shuffle-split + + + +-4202570559893519635->8372780637997478405 + + + + + +3079101795044967880 + +simple-shuffle-split + + + +-4202570559893519635->3079101795044967880 + + + + + +4201296338089766055 + +simple-shuffle-split + + + +-4202570559893519635->4201296338089766055 + + + + + +7439015692756806890 + +simple-shuffle-split + + + +-4202570559893519635->7439015692756806890 + + + + + +7293245819212519465 + +simple-shuffle-split + + + +-4202570559893519635->7293245819212519465 + + + + + +2079222347534809340 + +simple-shuffle-split + + + +-4202570559893519635->2079222347534809340 + + + + + +6349523891231900171 + +simple-shuffle-split + + + +-4202570559893519635->6349523891231900171 + + + + + +-8230882730113865826 + +simple-shuffle-split + + + +-4202570559893519635->-8230882730113865826 + + + + + +-7809079340773080451 + +simple-shuffle-split + + + +-4202570559893519635->-7809079340773080451 + + + + + +-2474530536820928960 + +simple-shuffle-split + + + +-4202570559893519635->-2474530536820928960 + + + + + +8048744656100737215 + +simple-shuffle-split + + + +-4202570559893519635->8048744656100737215 + + + + + +7944335219741355746 + +simple-shuffle-split + + + +-4202570559893519635->7944335219741355746 + + + + + +367592698401632705 + +simple-shuffle-split + + + +-4202570559893519635->367592698401632705 + + + + + +-3575251210567989420 + +simple-shuffle-split + + + +-4202570559893519635->-3575251210567989420 + + + + + +-2537313552999353245 + +simple-shuffle-split + + + +-4202570559893519635->-2537313552999353245 + + + + + +-7800222625764625770 + +simple-shuffle-split + + + +-4202570559893519635->-7800222625764625770 + + + + + +2106954949713205621 + +simple-shuffle-split + + + +-4202570559893519635->2106954949713205621 + + + + + +-9179630265242974920 + +simple-shuffle-split + + + +-4202570559893519635->-9179630265242974920 + + + + + +-7015834146179027945 + +simple-shuffle-split + + + +-4202570559893519635->-7015834146179027945 + + + + + +-6084355700711270310 + +simple-shuffle-split + + + +-4202570559893519635->-6084355700711270310 + + + + + +-4202570559892437110 + +17 + + + +-491696868626252584->-4202570559892437110 + + + + + +-1393577554088563125 + +simple-shuffle-split + + + +-4202570559892437110->-1393577554088563125 + + + + + +6116561309507533116 + +simple-shuffle-split + + + +-4202570559892437110->6116561309507533116 + + + + + +-8377379541055252375 + +simple-shuffle-split + + + +-4202570559892437110->-8377379541055252375 + + + + + +8583405443493537066 + +simple-shuffle-split + + + +-4202570559892437110->8583405443493537066 + + + + + +-7097949457268237825 + +simple-shuffle-split + + + +-4202570559892437110->-7097949457268237825 + + + + + +1528145207011666816 + +simple-shuffle-split + + + +-4202570559892437110->1528145207011666816 + + + + + +3359708357082430141 + +simple-shuffle-split + + + +-4202570559892437110->3359708357082430141 + + + + + +3505456989407769566 + +simple-shuffle-split + + + +-4202570559892437110->3505456989407769566 + + + + + +-920222243153004125 + +simple-shuffle-split + + + +-4202570559892437110->-920222243153004125 + + + + + +7034060437072100500 + +simple-shuffle-split + + + +-4202570559892437110->7034060437072100500 + + + + + +1105557976937825025 + +simple-shuffle-split + + + +-4202570559892437110->1105557976937825025 + + + + + +52184934233174050 + +simple-shuffle-split + + + +-4202570559892437110->52184934233174050 + + + + + +-4240406132102720425 + +simple-shuffle-split + + + +-4202570559892437110->-4240406132102720425 + + + + + +2633517564721947000 + +simple-shuffle-split + + + +-4202570559892437110->2633517564721947000 + + + + + +-2782160070110272075 + +simple-shuffle-split + + + +-4202570559892437110->-2782160070110272075 + + + + + +-4312889269536894250 + +simple-shuffle-split + + + +-4202570559892437110->-4312889269536894250 + + + + + +3534454525792805307 + +simple-shuffle-split + + + +-4202570559892437110->3534454525792805307 + + + + + +-6875149715532537684 + +simple-shuffle-split + + + +-4202570559892437110->-6875149715532537684 + + + + + +7815780101328268057 + +simple-shuffle-split + + + +-4202570559892437110->7815780101328268057 + + + + + +-3839217702092991334 + +simple-shuffle-split + + + +-4202570559892437110->-3839217702092991334 + + + + + +6421158251661802607 + +simple-shuffle-split + + + +-4202570559892437110->6421158251661802607 + + + + + +6563603015227645232 + +simple-shuffle-split + + + +-4202570559892437110->6563603015227645232 + + + + + +-7854100422467840659 + +simple-shuffle-split + + + +-4202570559892437110->-7854100422467840659 + + + + + +-342871098721008818 + +simple-shuffle-split + + + +-4202570559892437110->-342871098721008818 + + + + + +-5280607259195937709 + +simple-shuffle-split + + + +-4202570559892437110->-5280607259195937709 + + + + + +-6407230737691217084 + +simple-shuffle-split + + + +-4202570559892437110->-6407230737691217084 + + + + + +-12954114385398159 + +simple-shuffle-split + + + +-4202570559892437110->-12954114385398159 + + + + + +-1047437420216500334 + +simple-shuffle-split + + + +-4202570559892437110->-1047437420216500334 + + + + + +9204043238379769607 + +simple-shuffle-split + + + +-4202570559892437110->9204043238379769607 + + + + + +8869117983326472232 + +simple-shuffle-split + + + +-4202570559892437110->8869117983326472232 + + + + + +2141212568077271141 + +simple-shuffle-split + + + +-4202570559892437110->2141212568077271141 + + + + + +574626964712823366 + +simple-shuffle-split + + + +-4202570559892437110->574626964712823366 + + + + + +-3624493851078700757 + +simple-shuffle-split + + + +-4202570559892437110->-3624493851078700757 + + + + + +5004958808158739484 + +simple-shuffle-split + + + +-4202570559892437110->5004958808158739484 + + + + + +-2237030454554810807 + +simple-shuffle-split + + + +-4202570559892437110->-2237030454554810807 + + + + + +-3285138533207406582 + +simple-shuffle-split + + + +-4202570559892437110->-3285138533207406582 + + + + + +8751900762420155359 + +simple-shuffle-split + + + +-4202570559892437110->8751900762420155359 + + + + + +6517169587500562784 + +simple-shuffle-split + + + +-4202570559892437110->6517169587500562784 + + + + + +-6715566062408196707 + +simple-shuffle-split + + + +-4202570559892437110->-6715566062408196707 + + + + + +-6569822032598108482 + +simple-shuffle-split + + + +-4202570559892437110->-6569822032598108482 + + + + + +7489333169067194243 + +simple-shuffle-split + + + +-4202570559892437110->7489333169067194243 + + + + + +6032587197503294068 + +simple-shuffle-split + + + +-4202570559892437110->6032587197503294068 + + + + + +685776732316116193 + +simple-shuffle-split + + + +-4202570559892437110->685776732316116193 + + + + + +8533812120450402818 + +simple-shuffle-split + + + +-4202570559892437110->8533812120450402818 + + + + + +1934437038139033143 + +simple-shuffle-split + + + +-4202570559892437110->1934437038139033143 + + + + + +-9027672942637425832 + +simple-shuffle-split + + + +-4202570559892437110->-9027672942637425832 + + + + + +-3784704254675328107 + +simple-shuffle-split + + + +-4202570559892437110->-3784704254675328107 + + + + + +2978920566541864118 + +simple-shuffle-split + + + +-4202570559892437110->2978920566541864118 + + + + + +-8700906811781719141 + +simple-shuffle-split + + + +-4202570559892437110->-8700906811781719141 + + + + + +-4202570559895684685 + +18 + + + +-7408227040647628775->-4202570559895684685 + + + + + +-4999527107595264868 + +simple-shuffle-split + + + +-4202570559895684685->-4999527107595264868 + + + + + +4969709886731396523 + +simple-shuffle-split + + + +-4202570559895684685->4969709886731396523 + + + + + +5328310920313589898 + +simple-shuffle-split + + + +-4202570559895684685->5328310920313589898 + + + + + +6373097327572112073 + +simple-shuffle-split + + + +-4202570559895684685->6373097327572112073 + + + + + +6330275645409165792 + +simple-shuffle-split + + + +-4202570559895684685->6330275645409165792 + + + + + +6520838046366307423 + +simple-shuffle-split + + + +-4202570559895684685->6520838046366307423 + + + + + +-9007677172844180674 + +simple-shuffle-split + + + +-4202570559895684685->-9007677172844180674 + + + + + +9222386601420645917 + +simple-shuffle-split + + + +-4202570559895684685->9222386601420645917 + + + + + +4531461257890406132 + +simple-shuffle-split + + + +-4202570559895684685->4531461257890406132 + + + + + +-3403571071961885693 + +simple-shuffle-split + + + +-4202570559895684685->-3403571071961885693 + + + + + +-2360197071201217918 + +simple-shuffle-split + + + +-4202570559895684685->-2360197071201217918 + + + + + +746686772352533857 + +simple-shuffle-split + + + +-4202570559895684685->746686772352533857 + + + + + +-436632482171542568 + +simple-shuffle-split + + + +-4202570559895684685->-436632482171542568 + + + + + +-8922999661677477193 + +simple-shuffle-split + + + +-4202570559895684685->-8922999661677477193 + + + + + +-9140845919424303818 + +simple-shuffle-split + + + +-4202570559895684685->-9140845919424303818 + + + + + +-6404035595180120043 + +simple-shuffle-split + + + +-4202570559895684685->-6404035595180120043 + + + + + +8424693806398724108 + +simple-shuffle-split + + + +-4202570559895684685->8424693806398724108 + + + + + +-8978408514002600933 + +simple-shuffle-split + + + +-4202570559895684685->-8978408514002600933 + + + + + +2645316713263944442 + +simple-shuffle-split + + + +-4202570559895684685->2645316713263944442 + + + + + +3689800198435057017 + +simple-shuffle-split + + + +-4202570559895684685->3689800198435057017 + + + + + +-6023917159247511408 + +simple-shuffle-split + + + +-4202570559895684685->-6023917159247511408 + + + + + +-6237298120395126833 + +simple-shuffle-split + + + +-4202570559895684685->-6237298120395126833 + + + + + +-3093414328619320658 + +simple-shuffle-split + + + +-4202570559895684685->-3093414328619320658 + + + + + +-859769094525093683 + +simple-shuffle-split + + + +-4202570559895684685->-859769094525093683 + + + + + +735041124370256292 + +simple-shuffle-split + + + +-4202570559895684685->735041124370256292 + + + + + +-5008401621895459149 + +simple-shuffle-split + + + +-4202570559895684685->-5008401621895459149 + + + + + +4959726977664441842 + +simple-shuffle-split + + + +-4202570559895684685->4959726977664441842 + + + + + +4092187568622481617 + +simple-shuffle-split + + + +-4202570559895684685->4092187568622481617 + + + + + +-4855322530051705208 + +simple-shuffle-split + + + +-4202570559895684685->-4855322530051705208 + + + + + +-3234428079280084633 + +simple-shuffle-split + + + +-4202570559895684685->-3234428079280084633 + + + + + +6510859669978751142 + +simple-shuffle-split + + + +-4202570559895684685->6510859669978751142 + + + + + +-308332866379220283 + +simple-shuffle-split + + + +-4202570559895684685->-308332866379220283 + + + + + +-3814093257581505092 + +simple-shuffle-split + + + +-4202570559895684685->-3814093257581505092 + + + + + +-2770719257981304117 + +simple-shuffle-split + + + +-4202570559895684685->-2770719257981304117 + + + + + +-1862703586879264342 + +simple-shuffle-split + + + +-4202570559895684685->-1862703586879264342 + + + + + +8747366467418249449 + +simple-shuffle-split + + + +-4202570559895684685->8747366467418249449 + + + + + +8962771940231885824 + +simple-shuffle-split + + + +-4202570559895684685->8962771940231885824 + + + + + +-8436168071007506817 + +simple-shuffle-split + + + +-4202570559895684685->-8436168071007506817 + + + + + +-6960332671088551842 + +simple-shuffle-split + + + +-4202570559895684685->-6960332671088551842 + + + + + +3082207792188293949 + +simple-shuffle-split + + + +-4202570559895684685->3082207792188293949 + + + + + +8022306480516882708 + +simple-shuffle-split + + + +-4202570559895684685->8022306480516882708 + + + + + +7153658738784475683 + +simple-shuffle-split + + + +-4202570559895684685->7153658738784475683 + + + + + +-1323849537732501342 + +simple-shuffle-split + + + +-4202570559895684685->-1323849537732501342 + + + + + +-8826201469829395583 + +simple-shuffle-split + + + +-4202570559895684685->-8826201469829395583 + + + + + +-798608303081992 + +simple-shuffle-split + + + +-4202570559895684685->-798608303081992 + + + + + +2678726406776020183 + +simple-shuffle-split + + + +-4202570559895684685->2678726406776020183 + + + + + +-5798781835377283242 + +simple-shuffle-split + + + +-4202570559895684685->-5798781835377283242 + + + + + +6977824086149921333 + +simple-shuffle-split + + + +-4202570559895684685->6977824086149921333 + + + + + +-8323223006732261076 + +simple-shuffle-split + + + +-4202570559895684685->-8323223006732261076 + + + + + +-4202570559894602160 + +19 + + + +-8584387631107793650->-4202570559894602160 + + + + + +-4361290628027064195 + +simple-shuffle-split + + + +-4202570559894602160->-4361290628027064195 + + + + + +-5480181371005071970 + +simple-shuffle-split + + + +-4202570559894602160->-5480181371005071970 + + + + + +-5271230444997454145 + +simple-shuffle-split + + + +-4202570559894602160->-5271230444997454145 + + + + + +-8459440716446966720 + +simple-shuffle-split + + + +-4202570559894602160->-8459440716446966720 + + + + + +3765743178776105 + +simple-shuffle-split + + + +-4202570559894602160->3765743178776105 + + + + + +-969781542124270870 + +simple-shuffle-split + + + +-4202570559894602160->-969781542124270870 + + + + + +-189823072113760245 + +simple-shuffle-split + + + +-4202570559894602160->-189823072113760245 + + + + + +7168371417681878780 + +simple-shuffle-split + + + +-4202570559894602160->7168371417681878780 + + + + + +-1655923121234460811 + +simple-shuffle-split + + + +-4202570559894602160->-1655923121234460811 + + + + + +7035587802897479830 + +simple-shuffle-split + + + +-4202570559894602160->7035587802897479830 + + + + + +4289516849909381655 + +simple-shuffle-split + + + +-4202570559894602160->4289516849909381655 + + + + + +-5599888682697819336 + +simple-shuffle-split + + + +-4202570559894602160->-5599888682697819336 + + + + + +3965047606275786689 + +simple-shuffle-split + + + +-4202570559894602160->3965047606275786689 + + + + + +-7668511170842841886 + +simple-shuffle-split + + + +-4202570559894602160->-7668511170842841886 + + + + + +-8567667911312565661 + +simple-shuffle-split + + + +-4202570559894602160->-8567667911312565661 + + + + + +-8944023141916414636 + +simple-shuffle-split + + + +-4202570559894602160->-8944023141916414636 + + + + + +2871648071461419821 + +simple-shuffle-split + + + +-4202570559894602160->2871648071461419821 + + + + + +707830650176076046 + +simple-shuffle-split + + + +-4202570559894602160->707830650176076046 + + + + + +7624917281835082287 + +simple-shuffle-split + + + +-4202570559894602160->7624917281835082287 + + + + + +7770682623808476912 + +simple-shuffle-split + + + +-4202570559894602160->7770682623808476912 + + + + + +7346559056942895321 + +simple-shuffle-split + + + +-4202570559894602160->7346559056942895321 + + + + + +6373011764919533146 + +simple-shuffle-split + + + +-4202570559894602160->6373011764919533146 + + + + + +-6456753713237585029 + +simple-shuffle-split + + + +-4202570559894602160->-6456753713237585029 + + + + + +2091689592548223596 + +simple-shuffle-split + + + +-4202570559894602160->2091689592548223596 + + + + + +3345275684338122021 + +simple-shuffle-split + + + +-4202570559894602160->3345275684338122021 + + + + + +-7147208146494776570 + +simple-shuffle-split + + + +-4202570559894602160->-7147208146494776570 + + + + + +-8190582144969151545 + +simple-shuffle-split + + + +-4202570559894602160->-8190582144969151545 + + + + + +575032014101573096 + +simple-shuffle-split + + + +-4202570559894602160->575032014101573096 + + + + + +7820185535783727921 + +simple-shuffle-split + + + +-4202570559894602160->7820185535783727921 + + + + + +-1443104012542205870 + +simple-shuffle-split + + + +-4202570559894602160->-1443104012542205870 + + + + + +-2415547444643256045 + +simple-shuffle-split + + + +-4202570559894602160->-2415547444643256045 + + + + + +4317678251414868996 + +simple-shuffle-split + + + +-4202570559894602160->4317678251414868996 + + + + + +3496962879308602973 + +simple-shuffle-split + + + +-4202570559894602160->3496962879308602973 + + + + + +1885982700875363198 + +simple-shuffle-split + + + +-4202570559894602160->1885982700875363198 + + + + + +903576471568651423 + +simple-shuffle-split + + + +-4202570559894602160->903576471568651423 + + + + + +-1129171083479222752 + +simple-shuffle-split + + + +-4202570559894602160->-1129171083479222752 + + + + + +8488714598573067273 + +simple-shuffle-split + + + +-4202570559894602160->8488714598573067273 + + + + + +7440910606008042698 + +simple-shuffle-split + + + +-4202570559894602160->7440910606008042698 + + + + + +6532899476453446123 + +simple-shuffle-split + + + +-4202570559894602160->6532899476453446123 + + + + + +4461619479762557148 + +simple-shuffle-split + + + +-4202570559894602160->4461619479762557148 + + + + + +-4253925299060700843 + +simple-shuffle-split + + + +-4202570559894602160->-4253925299060700843 + + + + + +-6425489856407604618 + +simple-shuffle-split + + + +-4202570559894602160->-6425489856407604618 + + + + + +-5786561912606787593 + +simple-shuffle-split + + + +-4202570559894602160->-5786561912606787593 + + + + + +-6711218917612289768 + +simple-shuffle-split + + + +-4202570559894602160->-6711218917612289768 + + + + + +2853449653125469857 + +simple-shuffle-split + + + +-4202570559894602160->2853449653125469857 + + + + + +-8086477262763053118 + +simple-shuffle-split + + + +-4202570559894602160->-8086477262763053118 + + + + + +-2501878474183893693 + +simple-shuffle-split + + + +-4202570559894602160->-2501878474183893693 + + + + + +-2275214144126059468 + +simple-shuffle-split + + + +-4202570559894602160->-2275214144126059468 + + + + + +1188201927724267789 + +simple-shuffle-split + + + +-4202570559894602160->1188201927724267789 + + + + + +-4202570559889189535 + +20 + + + +-5500450667769605209->-4202570559889189535 + + + + + +-5059713591180130306 + +simple-shuffle-split + + + +-4202570559889189535->-5059713591180130306 + + + + + +-3538987973062564131 + +simple-shuffle-split + + + +-4202570559889189535->-3538987973062564131 + + + + + +1868907299547856544 + +simple-shuffle-split + + + +-4202570559889189535->1868907299547856544 + + + + + +-4922843520047418081 + +simple-shuffle-split + + + +-4202570559889189535->-4922843520047418081 + + + + + +-694676843814286006 + +simple-shuffle-split + + + +-4202570559889189535->-694676843814286006 + + + + + +427517742427589769 + +simple-shuffle-split + + + +-4202570559889189535->427517742427589769 + + + + + +6308367514484586844 + +simple-shuffle-split + + + +-4202570559889189535->6308367514484586844 + + + + + +7844907532395751019 + +simple-shuffle-split + + + +-4202570559889189535->7844907532395751019 + + + + + +7236301947436815094 + +simple-shuffle-split + + + +-4202570559889189535->7236301947436815094 + + + + + +-1385382234702651947 + +simple-shuffle-split + + + +-4202570559889189535->-1385382234702651947 + + + + + +4660683066481038744 + +simple-shuffle-split + + + +-4202570559889189535->4660683066481038744 + + + + + +5695202625987788919 + +simple-shuffle-split + + + +-4202570559889189535->5695202625987788919 + + + + + +-6625650574147877822 + +simple-shuffle-split + + + +-4202570559889189535->-6625650574147877822 + + + + + +4314275136795057953 + +simple-shuffle-split + + + +-4202570559889189535->4314275136795057953 + + + + + +-904996418368944972 + +simple-shuffle-split + + + +-4202570559889189535->-904996418368944972 + + + + + +8944382277282004419 + +simple-shuffle-split + + + +-4202570559889189535->8944382277282004419 + + + + + +-152168639420908690 + +simple-shuffle-split + + + +-4202570559889189535->-152168639420908690 + + + + + +1385493021076723085 + +simple-shuffle-split + + + +-4202570559889189535->1385493021076723085 + + + + + +8116723519700234576 + +simple-shuffle-split + + + +-4202570559889189535->8116723519700234576 + + + + + +8448323692453086351 + +simple-shuffle-split + + + +-4202570559889189535->8448323692453086351 + + + + + +7708834553089358010 + +simple-shuffle-split + + + +-4202570559889189535->7708834553089358010 + + + + + +-768673660242799815 + +simple-shuffle-split + + + +-4202570559889189535->-768673660242799815 + + + + + +2441181481422867660 + +simple-shuffle-split + + + +-4202570559889189535->2441181481422867660 + + + + + +-6036326804014115365 + +simple-shuffle-split + + + +-4202570559889189535->-6036326804014115365 + + + + + +-6175283792830314906 + +simple-shuffle-split + + + +-4202570559889189535->-6175283792830314906 + + + + + +-6439785698869668731 + +simple-shuffle-split + + + +-4202570559889189535->-6439785698869668731 + + + + + +775747295967653960 + +simple-shuffle-split + + + +-4202570559889189535->775747295967653960 + + + + + +-7844810031478616281 + +simple-shuffle-split + + + +-4202570559889189535->-7844810031478616281 + + + + + +-1171454758973597006 + +simple-shuffle-split + + + +-4202570559889189535->-1171454758973597006 + + + + + +-9204169096674066031 + +simple-shuffle-split + + + +-4202570559889189535->-9204169096674066031 + + + + + +5360537711891593060 + +simple-shuffle-split + + + +-4202570559889189535->5360537711891593060 + + + + + +-2215100980650576781 + +simple-shuffle-split + + + +-4202570559889189535->-2215100980650576781 + + + + + +-4019355889552495074 + +simple-shuffle-split + + + +-4202570559889189535->-4019355889552495074 + + + + + +-2974855624966756099 + +simple-shuffle-split + + + +-4202570559889189535->-2974855624966756099 + + + + + +-6612688139068988224 + +simple-shuffle-split + + + +-4202570559889189535->-6612688139068988224 + + + + + +-3810404891492013249 + +simple-shuffle-split + + + +-4202570559889189535->-3810404891492013249 + + + + + +-5680343258218753174 + +simple-shuffle-split + + + +-4202570559889189535->-5680343258218753174 + + + + + +4136985642569913001 + +simple-shuffle-split + + + +-4202570559889189535->4136985642569913001 + + + + + +1322705641627562876 + +simple-shuffle-split + + + +-4202570559889189535->1322705641627562876 + + + + + +2367187998288013451 + +simple-shuffle-split + + + +-4202570559889189535->2367187998288013451 + + + + + +-1174393604916196074 + +simple-shuffle-split + + + +-4202570559889189535->-1174393604916196074 + + + + + +-8675620445952303115 + +simple-shuffle-split + + + +-4202570559889189535->-8675620445952303115 + + + + + +5810904536423553976 + +simple-shuffle-split + + + +-4202570559889189535->5810904536423553976 + + + + + +5546421004903744151 + +simple-shuffle-split + + + +-4202570559889189535->5546421004903744151 + + + + + +-4670431581761669790 + +simple-shuffle-split + + + +-4202570559889189535->-4670431581761669790 + + + + + +-1855833945508935615 + +simple-shuffle-split + + + +-4202570559889189535->-1855833945508935615 + + + + + +-7631977627352874540 + +simple-shuffle-split + + + +-4202570559889189535->-7631977627352874540 + + + + + +2769876041474266851 + +simple-shuffle-split + + + +-4202570559889189535->2769876041474266851 + + + + + +-5136726695656620658 + +simple-shuffle-split + + + +-4202570559889189535->-5136726695656620658 + + + + + +-4202570559888107010 + +21 + + + +-5332030766655223284->-4202570559888107010 + + + + + +-2896326221037548417 + +simple-shuffle-split + + + +-4202570559888107010->-2896326221037548417 + + + + + +-3939704761128648192 + +simple-shuffle-split + + + +-4202570559888107010->-3939704761128648192 + + + + + +8622045128323449149 + +simple-shuffle-split + + + +-4202570559888107010->8622045128323449149 + + + + + +-3103904467101723042 + +simple-shuffle-split + + + +-4202570559888107010->-3103904467101723042 + + + + + +-6898587583792977717 + +simple-shuffle-split + + + +-4202570559888107010->-6898587583792977717 + + + + + +-51696539686298692 + +simple-shuffle-split + + + +-4202570559888107010->-51696539686298692 + + + + + +6852480346339315433 + +simple-shuffle-split + + + +-4202570559888107010->6852480346339315433 + + + + + +-5845246289797216342 + +simple-shuffle-split + + + +-4202570559888107010->-5845246289797216342 + + + + + +-2493638213323474217 + +simple-shuffle-split + + + +-4202570559888107010->-2493638213323474217 + + + + + +-1625026760756214792 + +simple-shuffle-split + + + +-4202570559888107010->-1625026760756214792 + + + + + +-8951522844431015883 + +simple-shuffle-split + + + +-4202570559888107010->-8951522844431015883 + + + + + +-258907002931702442 + +simple-shuffle-split + + + +-4202570559888107010->-258907002931702442 + + + + + +2584313638331957283 + +simple-shuffle-split + + + +-4202570559888107010->2584313638331957283 + + + + + +-7265096771664208108 + +simple-shuffle-split + + + +-4202570559888107010->-7265096771664208108 + + + + + +7939641765641320833 + +simple-shuffle-split + + + +-4202570559888107010->7939641765641320833 + + + + + +-2032915770529528158 + +simple-shuffle-split + + + +-4202570559888107010->-2032915770529528158 + + + + + +-6729441186959760657 + +simple-shuffle-split + + + +-4202570559888107010->-6729441186959760657 + + + + + +1107228292013339568 + +simple-shuffle-split + + + +-4202570559888107010->1107228292013339568 + + + + + +-3911543324150981651 + +simple-shuffle-split + + + +-4202570559888107010->-3911543324150981651 + + + + + +-6586660714851723826 + +simple-shuffle-split + + + +-4202570559888107010->-6586660714851723826 + + + + + +387531944478602299 + +simple-shuffle-split + + + +-4202570559888107010->387531944478602299 + + + + + +-2824362321194014932 + +simple-shuffle-split + + + +-4202570559888107010->-2824362321194014932 + + + + + +567796597906801049 + +simple-shuffle-split + + + +-4202570559888107010->567796597906801049 + + + + + +1423159941966510874 + +simple-shuffle-split + + + +-4202570559888107010->1423159941966510874 + + + + + +3034680462950520199 + +simple-shuffle-split + + + +-4202570559888107010->3034680462950520199 + + + + + +-6719462818331743576 + +simple-shuffle-split + + + +-4202570559888107010->-6719462818331743576 + + + + + +5645784816305451749 + +simple-shuffle-split + + + +-4202570559888107010->5645784816305451749 + + + + + +5836964808987263174 + +simple-shuffle-split + + + +-4202570559888107010->5836964808987263174 + + + + + +-7366811711852604717 + +simple-shuffle-split + + + +-4202570559888107010->-7366811711852604717 + + + + + +-8493435234688108092 + +simple-shuffle-split + + + +-4202570559888107010->-8493435234688108092 + + + + + +2882205737139473649 + +simple-shuffle-split + + + +-4202570559888107010->2882205737139473649 + + + + + +1834401664727405074 + +simple-shuffle-split + + + +-4202570559888107010->1834401664727405074 + + + + + +-4046547654399378849 + +simple-shuffle-split + + + +-4202570559888107010->-4046547654399378849 + + + + + +-5091030013328801824 + +simple-shuffle-split + + + +-4202570559888107010->-5091030013328801824 + + + + + +-1378802478178352099 + +simple-shuffle-split + + + +-4202570559888107010->-1378802478178352099 + + + + + +-3613551494460935874 + +simple-shuffle-split + + + +-4202570559888107010->-3613551494460935874 + + + + + +-761864190480856149 + +simple-shuffle-split + + + +-4202570559888107010->-761864190480856149 + + + + + +-616097714523553124 + +simple-shuffle-split + + + +-4202570559888107010->-616097714523553124 + + + + + +3026415783360563401 + +simple-shuffle-split + + + +-4202570559888107010->3026415783360563401 + + + + + +214968420569292426 + +simple-shuffle-split + + + +-4202570559888107010->214968420569292426 + + + + + +6026880236742944951 + +simple-shuffle-split + + + +-4202570559888107010->6026880236742944951 + + + + + +4439920901329813976 + +simple-shuffle-split + + + +-4202570559888107010->4439920901329813976 + + + + + +-8269548395809716715 + +simple-shuffle-split + + + +-4202570559888107010->-8269548395809716715 + + + + + +7070254227527062326 + +simple-shuffle-split + + + +-4202570559888107010->7070254227527062326 + + + + + +-6754682678991431165 + +simple-shuffle-split + + + +-4202570559888107010->-6754682678991431165 + + + + + +-7226173234478326540 + +simple-shuffle-split + + + +-4202570559888107010->-7226173234478326540 + + + + + +-4292265175838125215 + +simple-shuffle-split + + + +-4202570559888107010->-4292265175838125215 + + + + + +-5706879823683870590 + +simple-shuffle-split + + + +-4202570559888107010->-5706879823683870590 + + + + + +-7913733734018333233 + +simple-shuffle-split + + + +-4202570559888107010->-7913733734018333233 + + + + + +-4202570559891354585 + +22 + + + +-1663411785097453859->-4202570559891354585 + + + + + +-4923351608660794272 + +simple-shuffle-split + + + +-4202570559891354585->-4923351608660794272 + + + + + +-3950912682565726497 + +simple-shuffle-split + + + +-4202570559891354585->-3950912682565726497 + + + + + +7655488486947663294 + +simple-shuffle-split + + + +-4202570559891354585->7655488486947663294 + + + + + +-1045986255288955747 + +simple-shuffle-split + + + +-4202570559891354585->-1045986255288955747 + + + + + +521722753064620828 + +simple-shuffle-split + + + +-4202570559891354585->521722753064620828 + + + + + +-7956893847309398997 + +simple-shuffle-split + + + +-4202570559891354585->-7956893847309398997 + + + + + +-6974487624653720822 + +simple-shuffle-split + + + +-4202570559891354585->-6974487624653720822 + + + + + +4733729393398606153 + +simple-shuffle-split + + + +-4202570559891354585->4733729393398606153 + + + + + +5706168329418263128 + +simple-shuffle-split + + + +-4202570559891354585->5706168329418263128 + + + + + +-3477286169361220297 + +simple-shuffle-split + + + +-4202570559891354585->-3477286169361220297 + + + + + +-1314593870209461322 + +simple-shuffle-split + + + +-4202570559891354585->-1314593870209461322 + + + + + +1882525018418876053 + +simple-shuffle-split + + + +-4202570559891354585->1882525018418876053 + + + + + +1664683301076346228 + +simple-shuffle-split + + + +-4202570559891354585->1664683301076346228 + + + + + +4497030593911798403 + +simple-shuffle-split + + + +-4202570559891354585->4497030593911798403 + + + + + +-2938815174565533438 + +simple-shuffle-split + + + +-4202570559891354585->-2938815174565533438 + + + + + +6959449234391849953 + +simple-shuffle-split + + + +-4202570559891354585->6959449234391849953 + + + + + +1194932699260262160 + +simple-shuffle-split + + + +-4202570559891354585->1194932699260262160 + + + + + +-7506527031628162481 + +simple-shuffle-split + + + +-4202570559891354585->-7506527031628162481 + + + + + +-5938850860570615506 + +simple-shuffle-split + + + +-4202570559891354585->-5938850860570615506 + + + + + +-4895459036113764531 + +simple-shuffle-split + + + +-4202570559891354585->-4895459036113764531 + + + + + +5086091256136466060 + +simple-shuffle-split + + + +-4202570559891354585->5086091256136466060 + + + + + +-1731218926432692581 + +simple-shuffle-split + + + +-4202570559891354585->-1731218926432692581 + + + + + +-766566939613686406 + +simple-shuffle-split + + + +-4202570559891354585->-766566939613686406 + + + + + +-341424509669940231 + +simple-shuffle-split + + + +-4202570559891354585->-341424509669940231 + + + + + +-7699655287504900856 + +simple-shuffle-split + + + +-4202570559891354585->-7699655287504900856 + + + + + +5018333618329145319 + +simple-shuffle-split + + + +-4202570559891354585->5018333618329145319 + + + + + +4801632114246157094 + +simple-shuffle-split + + + +-4202570559891354585->4801632114246157094 + + + + + +-1963488869103635131 + +simple-shuffle-split + + + +-4202570559891354585->-1963488869103635131 + + + + + +7853803715967710244 + +simple-shuffle-split + + + +-4202570559891354585->7853803715967710244 + + + + + +-8421393668414024397 + +simple-shuffle-split + + + +-4202570559891354585->-8421393668414024397 + + + + + +-9209102716499905422 + +simple-shuffle-split + + + +-4202570559891354585->-9209102716499905422 + + + + + +692518619872159569 + +simple-shuffle-split + + + +-4202570559891354585->692518619872159569 + + + + + +7237619167383541376 + +simple-shuffle-split + + + +-4202570559891354585->7237619167383541376 + + + + + +7091852701454749951 + +simple-shuffle-split + + + +-4202570559891354585->7091852701454749951 + + + + + +8147808806910012126 + +simple-shuffle-split + + + +-4202570559891354585->8147808806910012126 + + + + + +8990366271234247101 + +simple-shuffle-split + + + +-4202570559891354585->8990366271234247101 + + + + + +-7227086359397401540 + +simple-shuffle-split + + + +-4202570559891354585->-7227086359397401540 + + + + + +-6806409164097795765 + +simple-shuffle-split + + + +-4202570559891354585->-6806409164097795765 + + + + + +-5824002932574072790 + +simple-shuffle-split + + + +-4202570559891354585->-5824002932574072790 + + + + + +-1442185845864075415 + +simple-shuffle-split + + + +-4202570559891354585->-1442185845864075415 + + + + + +-468638552784168840 + +simple-shuffle-split + + + +-4202570559891354585->-468638552784168840 + + + + + +-8424025024117636265 + +simple-shuffle-split + + + +-4202570559891354585->-8424025024117636265 + + + + + +1393266425878615510 + +simple-shuffle-split + + + +-4202570559891354585->1393266425878615510 + + + + + +3490440397308207285 + +simple-shuffle-split + + + +-4202570559891354585->3490440397308207285 + + + + + +2777121893051066260 + +simple-shuffle-split + + + +-4202570559891354585->2777121893051066260 + + + + + +3740665601496400035 + +simple-shuffle-split + + + +-4202570559891354585->3740665601496400035 + + + + + +6984052447520991010 + +simple-shuffle-split + + + +-4202570559891354585->6984052447520991010 + + + + + +7960921375431606785 + +simple-shuffle-split + + + +-4202570559891354585->7960921375431606785 + + + + + +1687289387074335792 + +simple-shuffle-split + + + +-4202570559891354585->1687289387074335792 + + + + + +-4202570559890272060 + +23 + + + +6753518853319697266->-4202570559890272060 + + + + + +-3066390384665253455 + +simple-shuffle-split + + + +-4202570559890272060->-3066390384665253455 + + + + + +5411117864173724370 + +simple-shuffle-split + + + +-4202570559890272060->5411117864173724370 + + + + + +4434248937406254995 + +simple-shuffle-split + + + +-4202570559890272060->4434248937406254995 + + + + + +2813355614002150020 + +simple-shuffle-split + + + +-4202570559890272060->2813355614002150020 + + + + + +-6931896946889560155 + +simple-shuffle-split + + + +-4202570559890272060->-6931896946889560155 + + + + + +967223098131784070 + +simple-shuffle-split + + + +-4202570559890272060->967223098131784070 + + + + + +1114097933383668295 + +simple-shuffle-split + + + +-4202570559890272060->1114097933383668295 + + + + + +-1699659099516841880 + +simple-shuffle-split + + + +-4202570559890272060->-1699659099516841880 + + + + + +5739543564280470361 + +simple-shuffle-split + + + +-4202570559890272060->5739543564280470361 + + + + + +-4674525829659598630 + +simple-shuffle-split + + + +-4202570559890272060->-4674525829659598630 + + + + + +-5710112963197068805 + +simple-shuffle-split + + + +-4202570559890272060->-5710112963197068805 + + + + + +1801129726996889836 + +simple-shuffle-split + + + +-4202570559890272060->1801129726996889836 + + + + + +8511945258326802861 + +simple-shuffle-split + + + +-4202570559890272060->8511945258326802861 + + + + + +8729733880398822286 + +simple-shuffle-split + + + +-4202570559890272060->8729733880398822286 + + + + + +7106917565802740911 + +simple-shuffle-split + + + +-4202570559890272060->7106917565802740911 + + + + + +-2634912434131576464 + +simple-shuffle-split + + + +-4202570559890272060->-2634912434131576464 + + + + + +1260190108635870785 + +simple-shuffle-split + + + +-4202570559890272060->1260190108635870785 + + + + + +912758783192327010 + +simple-shuffle-split + + + +-4202570559890272060->912758783192327010 + + + + + +-471377490525160221 + +simple-shuffle-split + + + +-4202570559890272060->-471377490525160221 + + + + + +-2094190472936087596 + +simple-shuffle-split + + + +-4202570559890272060->-2094190472936087596 + + + + + +-1947624202169039371 + +simple-shuffle-split + + + +-4202570559890272060->-1947624202169039371 + + + + + +4834141449538341654 + +simple-shuffle-split + + + +-4202570559890272060->4834141449538341654 + + + + + +-4982041670001794921 + +simple-shuffle-split + + + +-4202570559890272060->-4982041670001794921 + + + + + +1798209393923621304 + +simple-shuffle-split + + + +-4202570559890272060->1798209393923621304 + + + + + +754835437555138729 + +simple-shuffle-split + + + +-4202570559890272060->754835437555138729 + + + + + +8330505878823518570 + +simple-shuffle-split + + + +-4202570559890272060->8330505878823518570 + + + + + +-3474279315751989621 + +simple-shuffle-split + + + +-4202570559890272060->-3474279315751989621 + + + + + +-4447790321901080196 + +simple-shuffle-split + + + +-4202570559890272060->-4447790321901080196 + + + + + +-4879290354035955971 + +simple-shuffle-split + + + +-4202570559890272060->-4879290354035955971 + + + + + +3750144443833431070 + +simple-shuffle-split + + + +-4202570559890272060->3750144443833431070 + + + + + +2275685092606437695 + +simple-shuffle-split + + + +-4202570559890272060->2275685092606437695 + + + + + +-7767941282743909696 + +simple-shuffle-split + + + +-4202570559890272060->-7767941282743909696 + + + + + +-3664294335172843631 + +simple-shuffle-split + + + +-4202570559890272060->-3664294335172843631 + + + + + +-5185039469640003406 + +simple-shuffle-split + + + +-4202570559890272060->-5185039469640003406 + + + + + +-6233111224781098381 + +simple-shuffle-split + + + +-4202570559890272060->-6233111224781098381 + + + + + +1738921079897382244 + +simple-shuffle-split + + + +-4202570559890272060->1738921079897382244 + + + + + +-8077299428393495931 + +simple-shuffle-split + + + +-4202570559890272060->-8077299428393495931 + + + + + +7702078899038430310 + +simple-shuffle-split + + + +-4202570559890272060->7702078899038430310 + + + + + +-2266048574279781081 + +simple-shuffle-split + + + +-4202570559890272060->-2266048574279781081 + + + + + +5235231358632037960 + +simple-shuffle-split + + + +-4202570559890272060->5235231358632037960 + + + + + +2359664826952630585 + +simple-shuffle-split + + + +-4202570559890272060->2359664826952630585 + + + + + +-7490858488354222406 + +simple-shuffle-split + + + +-4202570559890272060->-7490858488354222406 + + + + + +-7974564874227436581 + +simple-shuffle-split + + + +-4202570559890272060->-7974564874227436581 + + + + + +654891153758522060 + +simple-shuffle-split + + + +-4202570559890272060->654891153758522060 + + + + + +-2084212103199564915 + +simple-shuffle-split + + + +-4202570559890272060->-2084212103199564915 + + + + + +-1938462336942278290 + +simple-shuffle-split + + + +-4202570559890272060->-1938462336942278290 + + + + + +3394165401896321679 + +simple-shuffle-split + + + +-4202570559890272060->3394165401896321679 + + + + + +2275291358589818704 + +simple-shuffle-split + + + +-4202570559890272060->2275291358589818704 + + + + + +147483825029438753 + +simple-shuffle-split + + + +-4202570559890272060->147483825029438753 + + + + + +-4202570559902179835 + +24 + + + +8470169458848030491->-4202570559902179835 + + + + + +5995923826237597890 + +simple-shuffle-split + + + +-4202570559902179835->5995923826237597890 + + + + + +8736130909828188065 + +simple-shuffle-split + + + +-4202570559902179835->8736130909828188065 + + + + + +2998878464120799540 + +simple-shuffle-split + + + +-4202570559902179835->2998878464120799540 + + + + + +-5010544406200074685 + +simple-shuffle-split + + + +-4202570559902179835->-5010544406200074685 + + + + + +-8985553183631030410 + +simple-shuffle-split + + + +-4202570559902179835->-8985553183631030410 + + + + + +2022009538427194965 + +simple-shuffle-split + + + +-4202570559902179835->2022009538427194965 + + + + + +-2056894835390313960 + +simple-shuffle-split + + + +-4202570559902179835->-2056894835390313960 + + + + + +-2202661268098578185 + +simple-shuffle-split + + + +-4202570559902179835->-2202661268098578185 + + + + + +5060671597911452106 + +simple-shuffle-split + + + +-4202570559902179835->5060671597911452106 + + + + + +-3568744744482841335 + +simple-shuffle-split + + + +-4202570559902179835->-3568744744482841335 + + + + + +-6531805814168794660 + +simple-shuffle-split + + + +-4202570559902179835->-6531805814168794660 + + + + + +-5487322364521822485 + +simple-shuffle-split + + + +-4202570559902179835->-5487322364521822485 + + + + + +6678305913371745406 + +simple-shuffle-split + + + +-4202570559902179835->6678305913371745406 + + + + + +7604057291639353181 + +simple-shuffle-split + + + +-4202570559902179835->7604057291639353181 + + + + + +4084973661638241056 + +simple-shuffle-split + + + +-4202570559902179835->4084973661638241056 + + + + + +5634931922622268831 + +simple-shuffle-split + + + +-4202570559902179835->5634931922622268831 + + + + + +3393644815770713906 + +simple-shuffle-split + + + +-4202570559902179835->3393644815770713906 + + + + + +3790744854084964881 + +simple-shuffle-split + + + +-4202570559902179835->3790744854084964881 + + + + + +-390241409878844444 + +simple-shuffle-split + + + +-4202570559902179835->-390241409878844444 + + + + + +2275611446760793331 + +simple-shuffle-split + + + +-4202570559902179835->2275611446760793331 + + + + + +-3365088579383744794 + +simple-shuffle-split + + + +-4202570559902179835->-3365088579383744794 + + + + + +-1879112792699207419 + +simple-shuffle-split + + + +-4202570559902179835->-1879112792699207419 + + + + + +-5924080906071941944 + +simple-shuffle-split + + + +-4202570559902179835->-5924080906071941944 + + + + + +-5628780061627079769 + +simple-shuffle-split + + + +-4202570559902179835->-5628780061627079769 + + + + + +-8517414290680849094 + +simple-shuffle-split + + + +-4202570559902179835->-8517414290680849094 + + + + + +-8663179631476456519 + +simple-shuffle-split + + + +-4202570559902179835->-8663179631476456519 + + + + + +6857503357280599372 + +simple-shuffle-split + + + +-4202570559902179835->6857503357280599372 + + + + + +8343053033217389147 + +simple-shuffle-split + + + +-4202570559902179835->8343053033217389147 + + + + + +5449968583171287022 + +simple-shuffle-split + + + +-4202570559902179835->5449968583171287022 + + + + + +6503327749672118797 + +simple-shuffle-split + + + +-4202570559902179835->6503327749672118797 + + + + + +217787584305579472 + +simple-shuffle-split + + + +-4202570559902179835->217787584305579472 + + + + + +1143516597329560847 + +simple-shuffle-split + + + +-4202570559902179835->1143516597329560847 + + + + + +7146408473977021922 + +simple-shuffle-split + + + +-4202570559902179835->7146408473977021922 + + + + + +364661246741218497 + +simple-shuffle-split + + + +-4202570559902179835->364661246741218497 + + + + + +-3066874605156147628 + +simple-shuffle-split + + + +-4202570559902179835->-3066874605156147628 + + + + + +-2744165197116753053 + +simple-shuffle-split + + + +-4202570559902179835->-2744165197116753053 + + + + + +2676746851370505622 + +simple-shuffle-split + + + +-4202570559902179835->2676746851370505622 + + + + + +-4114982725966597003 + +simple-shuffle-split + + + +-4202570559902179835->-4114982725966597003 + + + + + +-1598398031460905928 + +simple-shuffle-split + + + +-4202570559902179835->-1598398031460905928 + + + + + +-113547293783224553 + +simple-shuffle-split + + + +-4202570559902179835->-113547293783224553 + + + + + +-4157385808824800278 + +simple-shuffle-split + + + +-4202570559902179835->-4157385808824800278 + + + + + +-2567308919195046103 + +simple-shuffle-split + + + +-4202570559902179835->-2567308919195046103 + + + + + +-6680793832097383428 + +simple-shuffle-split + + + +-4202570559902179835->-6680793832097383428 + + + + + +-5632989839515038453 + +simple-shuffle-split + + + +-4202570559902179835->-5632989839515038453 + + + + + +8619747142748577438 + +simple-shuffle-split + + + +-4202570559902179835->8619747142748577438 + + + + + +1170598450161775997 + +simple-shuffle-split + + + +-4202570559902179835->1170598450161775997 + + + + + +7216627392361786688 + +simple-shuffle-split + + + +-4202570559902179835->7216627392361786688 + + + + + +8343287195529033663 + +simple-shuffle-split + + + +-4202570559902179835->8343287195529033663 + + + + + +1910055769872256338 + +simple-shuffle-split + + + +-4202570559902179835->1910055769872256338 + + + + + +-4202570559901097310 + +25 + + + +7457217997785096016->-4202570559901097310 + + + + + +-1741569752342307677 + +simple-shuffle-split + + + +-4202570559901097310->-1741569752342307677 + + + + + +6707014549213152148 + +simple-shuffle-split + + + +-4202570559901097310->6707014549213152148 + + + + + +-7651133118211551743 + +simple-shuffle-split + + + +-4202570559901097310->-7651133118211551743 + + + + + +-8703366091727748318 + +simple-shuffle-split + + + +-4202570559901097310->-8703366091727748318 + + + + + +995378323287150423 + +simple-shuffle-split + + + +-4202570559901097310->995378323287150423 + + + + + +1138798804532528248 + +simple-shuffle-split + + + +-4202570559901097310->1138798804532528248 + + + + + +6359619510700424373 + +simple-shuffle-split + + + +-4202570559901097310->6359619510700424373 + + + + + +4192485023931056598 + +simple-shuffle-split + + + +-4202570559901097310->4192485023931056598 + + + + + +-4047083629636983477 + +simple-shuffle-split + + + +-4202570559901097310->-4047083629636983477 + + + + + +-3336112252949850052 + +simple-shuffle-split + + + +-4202570559901097310->-3336112252949850052 + + + + + +-1580236089230591127 + +simple-shuffle-split + + + +-4202570559901097310->-1580236089230591127 + + + + + +-2993746873863148502 + +simple-shuffle-split + + + +-4202570559901097310->-2993746873863148502 + + + + + +-8555669404345336577 + +simple-shuffle-split + + + +-4202570559901097310->-8555669404345336577 + + + + + +7135036429757416064 + +simple-shuffle-split + + + +-4202570559901097310->7135036429757416064 + + + + + +-4261845769901365827 + +simple-shuffle-split + + + +-4202570559901097310->-4261845769901365827 + + + + + +-7582961617179789602 + +simple-shuffle-split + + + +-4202570559901097310->-7582961617179789602 + + + + + +4336196443053145939 + +simple-shuffle-split + + + +-4202570559901097310->4336196443053145939 + + + + + +3282822238651946564 + +simple-shuffle-split + + + +-4202570559901097310->3282822238651946564 + + + + + +6911779000428139889 + +simple-shuffle-split + + + +-4202570559901097310->6911779000428139889 + + + + + +5382989607097620114 + +simple-shuffle-split + + + +-4202570559901097310->5382989607097620114 + + + + + +-104274645800903161 + +simple-shuffle-split + + + +-4202570559901097310->-104274645800903161 + + + + + +-2268034439136338136 + +simple-shuffle-split + + + +-4202570559901097310->-2268034439136338136 + + + + + +1316580840572601189 + +simple-shuffle-split + + + +-4202570559901097310->1316580840572601189 + + + + + +272076044346687814 + +simple-shuffle-split + + + +-4202570559901097310->272076044346687814 + + + + + +1407949456721687739 + +simple-shuffle-split + + + +-4202570559901097310->1407949456721687739 + + + + + +8918088318100772780 + +simple-shuffle-split + + + +-4202570559901097310->8918088318100772780 + + + + + +-5464031639798759911 + +simple-shuffle-split + + + +-4202570559901097310->-5464031639798759911 + + + + + +-5247330108072413286 + +simple-shuffle-split + + + +-4202570559901097310->-5247330108072413286 + + + + + +7602287054003199855 + +simple-shuffle-split + + + +-4202570559901097310->7602287054003199855 + + + + + +4403263036694085680 + +simple-shuffle-split + + + +-4202570559901097310->4403263036694085680 + + + + + +7870284326644253805 + +simple-shuffle-split + + + +-4202570559901097310->7870284326644253805 + + + + + +8645661054729226830 + +simple-shuffle-split + + + +-4202570559901097310->8645661054729226830 + + + + + +4324487403022210691 + +simple-shuffle-split + + + +-4202570559901097310->4324487403022210691 + + + + + +-5564922678874652300 + +simple-shuffle-split + + + +-4202570559901097310->-5564922678874652300 + + + + + +-8762467892456454175 + +simple-shuffle-split + + + +-4202570559901097310->-8762467892456454175 + + + + + +-222878983567018750 + +simple-shuffle-split + + + +-4202570559901097310->-222878983567018750 + + + + + +467821865704622391 + +simple-shuffle-split + + + +-4202570559901097310->467821865704622391 + + + + + +136253450494064216 + +simple-shuffle-split + + + +-4202570559901097310->136253450494064216 + + + + + +-7101458147496154475 + +simple-shuffle-split + + + +-4202570559901097310->-7101458147496154475 + + + + + +1515625864989962166 + +simple-shuffle-split + + + +-4202570559901097310->1515625864989962166 + + + + + +-5087751021956503509 + +simple-shuffle-split + + + +-4202570559901097310->-5087751021956503509 + + + + + +2878453774559801116 + +simple-shuffle-split + + + +-4202570559901097310->2878453774559801116 + + + + + +8659196527181632841 + +simple-shuffle-split + + + +-4202570559901097310->8659196527181632841 + + + + + +-4105344799300825334 + +simple-shuffle-split + + + +-4202570559901097310->-4105344799300825334 + + + + + +-1190613698393195809 + +simple-shuffle-split + + + +-4202570559901097310->-1190613698393195809 + + + + + +-2168456074941665184 + +simple-shuffle-split + + + +-4202570559901097310->-2168456074941665184 + + + + + +-8049292526000265059 + +simple-shuffle-split + + + +-4202570559901097310->-8049292526000265059 + + + + + +-1407315231262688834 + +simple-shuffle-split + + + +-4202570559901097310->-1407315231262688834 + + + + + +-5700995622656904909 + +simple-shuffle-split + + + +-4202570559901097310->-5700995622656904909 + + + + + +-4202570559904344885 + +26 + + + +-5580019870367564975->-4202570559904344885 + + + + + +-5038466543536177676 + +simple-shuffle-split + + + +-4202570559904344885->-5038466543536177676 + + + + + +-3965064828788072701 + +simple-shuffle-split + + + +-4202570559904344885->-3965064828788072701 + + + + + +-796067390339340926 + +simple-shuffle-split + + + +-4202570559904344885->-796067390339340926 + + + + + +259621023484209249 + +simple-shuffle-split + + + +-4202570559904344885->259621023484209249 + + + + + +8920021036213978840 + +simple-shuffle-split + + + +-4202570559904344885->8920021036213978840 + + + + + +-8483344498547619401 + +simple-shuffle-split + + + +-4202570559904344885->-8483344498547619401 + + + + + +-6417892052090158026 + +simple-shuffle-split + + + +-4202570559904344885->-6417892052090158026 + + + + + +-4179822429687382251 + +simple-shuffle-split + + + +-4202570559904344885->-4179822429687382251 + + + + + +-5526071592311263332 + +simple-shuffle-split + + + +-4202570559904344885->-5526071592311263332 + + + + + +-5105389890936393557 + +simple-shuffle-split + + + +-4202570559904344885->-5105389890936393557 + + + + + +-2932739384969944182 + +simple-shuffle-split + + + +-4202570559904344885->-2932739384969944182 + + + + + +9145617873162001865 + +simple-shuffle-split + + + +-4202570559904344885->9145617873162001865 + + + + + +7360920541637365984 + +simple-shuffle-split + + + +-4202570559904344885->7360920541637365984 + + + + + +8835383288251652959 + +simple-shuffle-split + + + +-4202570559904344885->8835383288251652959 + + + + + +-8492486739899743682 + +simple-shuffle-split + + + +-4202570559904344885->-8492486739899743682 + + + + + +-6451836927807521507 + +simple-shuffle-split + + + +-4202570559904344885->-6451836927807521507 + + + + + +2373597681405778084 + +simple-shuffle-split + + + +-4202570559904344885->2373597681405778084 + + + + + +3354895544766875059 + +simple-shuffle-split + + + +-4202570559904344885->3354895544766875059 + + + + + +4911789705267720434 + +simple-shuffle-split + + + +-4202570559904344885->4911789705267720434 + + + + + +6316441590306118609 + +simple-shuffle-split + + + +-4202570559904344885->6316441590306118609 + + + + + +-3397008040237917816 + +simple-shuffle-split + + + +-4202570559904344885->-3397008040237917816 + + + + + +-1163384088411986841 + +simple-shuffle-split + + + +-4202570559904344885->-1163384088411986841 + + + + + +-1846674863145291866 + +simple-shuffle-split + + + +-4202570559904344885->-1846674863145291866 + + + + + +-944179717246316091 + +simple-shuffle-split + + + +-4202570559904344885->-944179717246316091 + + + + + +7863236444028396300 + +simple-shuffle-split + + + +-4202570559904344885->7863236444028396300 + + + + + +-6205851716954074341 + +simple-shuffle-split + + + +-4202570559904344885->-6205851716954074341 + + + + + +-5241199648105653766 + +simple-shuffle-split + + + +-4202570559904344885->-5241199648105653766 + + + + + +-5388069941740813191 + +simple-shuffle-split + + + +-4202570559904344885->-5388069941740813191 + + + + + +3493737868525402000 + +simple-shuffle-split + + + +-4202570559904344885->3493737868525402000 + + + + + +-4631735014295950641 + +simple-shuffle-split + + + +-4202570559904344885->-4631735014295950641 + + + + + +6530060438400159150 + +simple-shuffle-split + + + +-4202570559904344885->6530060438400159150 + + + + + +6886368580317797325 + +simple-shuffle-split + + + +-4202570559904344885->6886368580317797325 + + + + + +-1548728614913080300 + +simple-shuffle-split + + + +-4202570559904344885->-1548728614913080300 + + + + + +-9067726388114526941 + +simple-shuffle-split + + + +-4202570559904344885->-9067726388114526941 + + + + + +900401078621932450 + +simple-shuffle-split + + + +-4202570559904344885->900401078621932450 + + + + + +2450359337354308225 + +simple-shuffle-split + + + +-4202570559904344885->2450359337354308225 + + + + + +-8381584877766325000 + +simple-shuffle-split + + + +-4202570559904344885->-8381584877766325000 + + + + + +-7408037585777603625 + +simple-shuffle-split + + + +-4202570559904344885->-7408037585777603625 + + + + + +5439480989815576150 + +simple-shuffle-split + + + +-4202570559904344885->5439480989815576150 + + + + + +-3108980741532995275 + +simple-shuffle-split + + + +-4202570559904344885->-3108980741532995275 + + + + + +6745024960371453116 + +simple-shuffle-split + + + +-4202570559904344885->6745024960371453116 + + + + + +6599259629500434891 + +simple-shuffle-split + + + +-4202570559904344885->6599259629500434891 + + + + + +-335540310859985750 + +simple-shuffle-split + + + +-4202570559904344885->-335540310859985750 + + + + + +-8813048551974065175 + +simple-shuffle-split + + + +-4202570559904344885->-8813048551974065175 + + + + + +-8855212455170848000 + +simple-shuffle-split + + + +-4202570559904344885->-8855212455170848000 + + + + + +425832416470392191 + +simple-shuffle-split + + + +-4202570559904344885->425832416470392191 + + + + + +1472929622296854366 + +simple-shuffle-split + + + +-4202570559904344885->1472929622296854366 + + + + + +5768157097700078141 + +simple-shuffle-split + + + +-4202570559904344885->5768157097700078141 + + + + + +3887445310551288516 + +simple-shuffle-split + + + +-4202570559904344885->3887445310551288516 + + + + + +-4202570559903262360 + +27 + + + +1729916036795262950->-4202570559903262360 + + + + + +8604617348867777141 + +simple-shuffle-split + + + +-4202570559903262360->8604617348867777141 + + + + + +7060175082728348566 + +simple-shuffle-split + + + +-4202570559903262360->7060175082728348566 + + + + + +6755978969659815191 + +simple-shuffle-split + + + +-4202570559903262360->6755978969659815191 + + + + + +5086452695504693816 + +simple-shuffle-split + + + +-4202570559903262360->5086452695504693816 + + + + + +-2512961549035590975 + +simple-shuffle-split + + + +-4202570559903262360->-2512961549035590975 + + + + + +-3566302836380318750 + +simple-shuffle-split + + + +-4202570559903262360->-3566302836380318750 + + + + + +-7872718535325869725 + +simple-shuffle-split + + + +-4202570559903262360->-7872718535325869725 + + + + + +-8798483906242345900 + +simple-shuffle-split + + + +-4202570559903262360->-8798483906242345900 + + + + + +-1468578046838781059 + +simple-shuffle-split + + + +-4202570559903262360->-1468578046838781059 + + + + + +-2957470705207124834 + +simple-shuffle-split + + + +-4202570559903262360->-2957470705207124834 + + + + + +-5614428255585862209 + +simple-shuffle-split + + + +-4202570559903262360->-5614428255585862209 + + + + + +-5550802954723617984 + +simple-shuffle-split + + + +-4202570559903262360->-5550802954723617984 + + + + + +5241114389059769641 + +simple-shuffle-split + + + +-4202570559903262360->5241114389059769641 + + + + + +2585265160303743466 + +simple-shuffle-split + + + +-4202570559903262360->2585265160303743466 + + + + + +1090819086193396491 + +simple-shuffle-split + + + +-4202570559903262360->1090819086193396491 + + + + + +-394030507489185284 + +simple-shuffle-split + + + +-4202570559903262360->-394030507489185284 + + + + + +4113469716384568357 + +simple-shuffle-split + + + +-4202570559903262360->4113469716384568357 + + + + + +3065648989715083782 + +simple-shuffle-split + + + +-4202570559903262360->3065648989715083782 + + + + + +2139915399671480007 + +simple-shuffle-split + + + +-4202570559903262360->2139915399671480007 + + + + + +-8399487272735292184 + +simple-shuffle-split + + + +-4202570559903262360->-8399487272735292184 + + + + + +-7072656448450718159 + +simple-shuffle-split + + + +-4202570559903262360->-7072656448450718159 + + + + + +-7432380555613506734 + +simple-shuffle-split + + + +-4202570559903262360->-7432380555613506734 + + + + + +-8973479904705891309 + +simple-shuffle-split + + + +-4202570559903262360->-8973479904705891309 + + + + + +8421030140207452932 + +simple-shuffle-split + + + +-4202570559903262360->8421030140207452932 + + + + + +-5369699232878898643 + +simple-shuffle-split + + + +-4202570559903262360->-5369699232878898643 + + + + + +-6858996734189228018 + +simple-shuffle-split + + + +-4202570559903262360->-6858996734189228018 + + + + + +428261545813756207 + +simple-shuffle-split + + + +-4202570559903262360->428261545813756207 + + + + + +9028753949775256048 + +simple-shuffle-split + + + +-4202570559903262360->9028753949775256048 + + + + + +295459513295415257 + +simple-shuffle-split + + + +-4202570559903262360->295459513295415257 + + + + + +-109427391133527718 + +simple-shuffle-split + + + +-4202570559903262360->-109427391133527718 + + + + + +-1145050838205948293 + +simple-shuffle-split + + + +-4202570559903262360->-1145050838205948293 + + + + + +5646736293867732332 + +simple-shuffle-split + + + +-4202570559903262360->5646736293867732332 + + + + + +-1470389414463316907 + +simple-shuffle-split + + + +-4202570559903262360->-1470389414463316907 + + + + + +-3020368940984134282 + +simple-shuffle-split + + + +-4202570559903262360->-3020368940984134282 + + + + + +5605762040156237559 + +simple-shuffle-split + + + +-4202570559903262360->5605762040156237559 + + + + + +-5995199348741216232 + +simple-shuffle-split + + + +-4202570559903262360->-5995199348741216232 + + + + + +3004521644214168993 + +simple-shuffle-split + + + +-4202570559903262360->3004521644214168993 + + + + + +2644812172650817218 + +simple-shuffle-split + + + +-4202570559903262360->2644812172650817218 + + + + + +-8361620835912909757 + +simple-shuffle-split + + + +-4202570559903262360->-8361620835912909757 + + + + + +8497739222820415284 + +simple-shuffle-split + + + +-4202570559903262360->8497739222820415284 + + + + + +5861727909640463709 + +simple-shuffle-split + + + +-4202570559903262360->5861727909640463709 + + + + + +-3955585988668052866 + +simple-shuffle-split + + + +-4202570559903262360->-3955585988668052866 + + + + + +-5500005809786092641 + +simple-shuffle-split + + + +-4202570559903262360->-5500005809786092641 + + + + + +2968607171537719584 + +simple-shuffle-split + + + +-4202570559903262360->2968607171537719584 + + + + + +-6919856353729398007 + +simple-shuffle-split + + + +-4202570559903262360->-6919856353729398007 + + + + + +519311121683468234 + +simple-shuffle-split + + + +-4202570559903262360->519311121683468234 + + + + + +7227812450224743659 + +simple-shuffle-split + + + +-4202570559903262360->7227812450224743659 + + + + + +6183328958454558684 + +simple-shuffle-split + + + +-4202570559903262360->6183328958454558684 + + + + + +3698959137933080325 + +simple-shuffle-split + + + +-4202570559903262360->3698959137933080325 + + + + + +-4202570559897849735 + +28 + + + +3638677798482865791->-4202570559897849735 + + + + + +5925664213485840886 + +simple-shuffle-split + + + +-4202570559897849735->5925664213485840886 + + + + + +8681683999765275861 + +simple-shuffle-split + + + +-4202570559897849735->8681683999765275861 + + + + + +2895657186813456536 + +simple-shuffle-split + + + +-4202570559897849735->2895657186813456536 + + + + + +5697941532884840311 + +simple-shuffle-split + + + +-4202570559897849735->5697941532884840311 + + + + + +-4550218554994135230 + +simple-shuffle-split + + + +-4202570559897849735->-4550218554994135230 + + + + + +-5251222663889842655 + +simple-shuffle-split + + + +-4202570559897849735->-5251222663889842655 + + + + + +-8792621140124309580 + +simple-shuffle-split + + + +-4202570559897849735->-8792621140124309580 + + + + + +-5593592582358937405 + +simple-shuffle-split + + + +-4202570559897849735->-5593592582358937405 + + + + + +-3477443014426234114 + +simple-shuffle-split + + + +-4202570559897849735->-3477443014426234114 + + + + + +-1313629062832699939 + +simple-shuffle-split + + + +-4202570559897849735->-1313629062832699939 + + + + + +-6461168746197626464 + +simple-shuffle-split + + + +-4202570559897849735->-6461168746197626464 + + + + + +-6669011349474027489 + +simple-shuffle-split + + + +-4202570559897849735->-6669011349474027489 + + + + + +2077880023070093386 + +simple-shuffle-split + + + +-4202570559897849735->2077880023070093386 + + + + + +3122362380821729161 + +simple-shuffle-split + + + +-4202570559897849735->3122362380821729161 + + + + + +-937310084744032164 + +simple-shuffle-split + + + +-4202570559897849735->-937310084744032164 + + + + + +107172274133429611 + +simple-shuffle-split + + + +-4202570559897849735->107172274133429611 + + + + + +3291227880397728102 + +simple-shuffle-split + + + +-4202570559897849735->3291227880397728102 + + + + + +6392632036094719877 + +simple-shuffle-split + + + +-4202570559897849735->6392632036094719877 + + + + + +8992673751426318152 + +simple-shuffle-split + + + +-4202570559897849735->8992673751426318152 + + + + + +2216717736553829927 + +simple-shuffle-split + + + +-4202570559897849735->2216717736553829927 + + + + + +7704254291358855602 + +simple-shuffle-split + + + +-4202570559897849735->7704254291358855602 + + + + + +259646999892514961 + +simple-shuffle-split + + + +-4202570559897849735->259646999892514961 + + + + + +7437115520122686052 + +simple-shuffle-split + + + +-4202570559897849735->7437115520122686052 + + + + + +6660611395198675827 + +simple-shuffle-split + + + +-4202570559897849735->6660611395198675827 + + + + + +-7475563045267895698 + +simple-shuffle-split + + + +-4202570559897849735->-7475563045267895698 + + + + + +-6427737742687294323 + +simple-shuffle-split + + + +-4202570559897849735->-6427737742687294323 + + + + + +-419292450384432048 + +simple-shuffle-split + + + +-4202570559897849735->-419292450384432048 + + + + + +-7826413541824066673 + +simple-shuffle-split + + + +-4202570559897849735->-7826413541824066673 + + + + + +-2299114499532796998 + +simple-shuffle-split + + + +-4202570559897849735->-2299114499532796998 + + + + + +-762577930130593223 + +simple-shuffle-split + + + +-4202570559897849735->-762577930130593223 + + + + + +5214845501530415052 + +simple-shuffle-split + + + +-4202570559897849735->5214845501530415052 + + + + + +-3263769937129735973 + +simple-shuffle-split + + + +-4202570559897849735->-3263769937129735973 + + + + + +1055394241889975318 + +simple-shuffle-split + + + +-4202570559897849735->1055394241889975318 + + + + + +200088534544296693 + +simple-shuffle-split + + + +-4202570559897849735->200088534544296693 + + + + + +-3169291610382306632 + +simple-shuffle-split + + + +-4202570559897849735->-3169291610382306632 + + + + + +8333761803684776343 + +simple-shuffle-split + + + +-4202570559897849735->8333761803684776343 + + + + + +5526165357774981218 + +simple-shuffle-split + + + +-4202570559897849735->5526165357774981218 + + + + + +5242823852501084993 + +simple-shuffle-split + + + +-4202570559897849735->5242823852501084993 + + + + + +1244838644930093268 + +simple-shuffle-split + + + +-4202570559897849735->1244838644930093268 + + + + + +-6275263008077484573 + +simple-shuffle-split + + + +-4202570559897849735->-6275263008077484573 + + + + + +-1250157022007712482 + +simple-shuffle-split + + + +-4202570559897849735->-1250157022007712482 + + + + + +-202335160297775107 + +simple-shuffle-split + + + +-4202570559897849735->-202335160297775107 + + + + + +4510661565374261184 + +simple-shuffle-split + + + +-4202570559897849735->4510661565374261184 + + + + + +5562894465763728959 + +simple-shuffle-split + + + +-4202570559897849735->5562894465763728959 + + + + + +2606508516418381418 + +simple-shuffle-split + + + +-4202570559897849735->2606508516418381418 + + + + + +-4243801692167211607 + +simple-shuffle-split + + + +-4202570559897849735->-4243801692167211607 + + + + + +-8759632828547001732 + +simple-shuffle-split + + + +-4202570559897849735->-8759632828547001732 + + + + + +-7223097358834314357 + +simple-shuffle-split + + + +-4202570559897849735->-7223097358834314357 + + + + + +-5229021741818739066 + +simple-shuffle-split + + + +-4202570559897849735->-5229021741818739066 + + + + + +-4202570559896767210 + +29 + + + +2612727456218275716->-4202570559896767210 + + + + + +-272741049218666025 + +simple-shuffle-split + + + +-4202570559896767210->-272741049218666025 + + + + + +-497106873314702600 + +simple-shuffle-split + + + +-4202570559896767210->-497106873314702600 + + + + + +2325037936148553525 + +simple-shuffle-split + + + +-4202570559896767210->2325037936148553525 + + + + + +775062942307134550 + +simple-shuffle-split + + + +-4202570559896767210->775062942307134550 + + + + + +5357941405872362275 + +simple-shuffle-split + + + +-4202570559896767210->5357941405872362275 + + + + + +3368411928006535700 + +simple-shuffle-split + + + +-4202570559896767210->3368411928006535700 + + + + + +6761025966748881025 + +simple-shuffle-split + + + +-4202570559896767210->6761025966748881025 + + + + + +6330384880165906850 + +simple-shuffle-split + + + +-4202570559896767210->6330384880165906850 + + + + + +6478078126746936191 + +simple-shuffle-split + + + +-4202570559896767210->6478078126746936191 + + + + + +5424740235828726016 + +simple-shuffle-split + + + +-4202570559896767210->5424740235828726016 + + + + + +-7748121100574324675 + +simple-shuffle-split + + + +-4202570559896767210->-7748121100574324675 + + + + + +8463873892559318366 + +simple-shuffle-split + + + +-4202570559896767210->8463873892559318366 + + + + + +-7458972949195241525 + +simple-shuffle-split + + + +-4202570559896767210->-7458972949195241525 + + + + + +-8432520205694463300 + +simple-shuffle-split + + + +-4202570559896767210->-8432520205694463300 + + + + + +-3234287094775229975 + +simple-shuffle-split + + + +-4202570559896767210->-3234287094775229975 + + + + + +-4207830947036433750 + +simple-shuffle-split + + + +-4202570559896767210->-4207830947036433750 + + + + + +-4139928299350252409 + +simple-shuffle-split + + + +-4202570559896767210->-4139928299350252409 + + + + + +-5681027611896592984 + +simple-shuffle-split + + + +-4202570559896767210->-5681027611896592984 + + + + + +8419413684591917541 + +simple-shuffle-split + + + +-4202570559896767210->8419413684591917541 + + + + + +-3092124228081330234 + +simple-shuffle-split + + + +-4202570559896767210->-3092124228081330234 + + + + + +300506510332519891 + +simple-shuffle-split + + + +-4202570559896767210->300506510332519891 + + + + + +-55747436659732284 + +simple-shuffle-split + + + +-4202570559896767210->-55747436659732284 + + + + + +5642550134643833841 + +simple-shuffle-split + + + +-4202570559896767210->5642550134643833841 + + + + + +2463198802763963666 + +simple-shuffle-split + + + +-4202570559896767210->2463198802763963666 + + + + + +-4627265585618549265 + +simple-shuffle-split + + + +-4202570559896767210->-4627265585618549265 + + + + + +2629966072257693360 + +simple-shuffle-split + + + +-4202570559896767210->2629966072257693360 + + + + + +-4438088943976714515 + +simple-shuffle-split + + + +-4202570559896767210->-4438088943976714515 + + + + + +-5411652998856400690 + +simple-shuffle-split + + + +-4202570559896767210->-5411652998856400690 + + + + + +-139012539128890565 + +simple-shuffle-split + + + +-4202570559896767210->-139012539128890565 + + + + + +-1146363209933603284 + +simple-shuffle-split + + + +-4202570559896767210->-1146363209933603284 + + + + + +2788730048942425241 + +simple-shuffle-split + + + +-4202570559896767210->2788730048942425241 + + + + + +1407882507375107610 + +simple-shuffle-split + + + +-4202570559896767210->1407882507375107610 + + + + + +-2866314360304451657 + +simple-shuffle-split + + + +-4202570559896767210->-2866314360304451657 + + + + + +4488555009518871768 + +simple-shuffle-split + + + +-4202570559896767210->4488555009518871768 + + + + + +8425454301161020693 + +simple-shuffle-split + + + +-4202570559896767210->8425454301161020693 + + + + + +8634387392422040118 + +simple-shuffle-split + + + +-4202570559896767210->8634387392422040118 + + + + + +-4679214365147333373 + +simple-shuffle-split + + + +-4202570559896767210->-4679214365147333373 + + + + + +2840891793935508468 + +simple-shuffle-split + + + +-4202570559896767210->2840891793935508468 + + + + + +5648627103614864993 + +simple-shuffle-split + + + +-4202570559896767210->5648627103614864993 + + + + + +-3634730845612547198 + +simple-shuffle-split + + + +-4202570559896767210->-3634730845612547198 + + + + + +-5156341980664311457 + +simple-shuffle-split + + + +-4202570559896767210->-5156341980664311457 + + + + + +-5655839772651424032 + +simple-shuffle-split + + + +-4202570559896767210->-5655839772651424032 + + + + + +203469232576772893 + +simple-shuffle-split + + + +-4202570559896767210->203469232576772893 + + + + + +5866675987922767934 + +simple-shuffle-split + + + +-4202570559896767210->5866675987922767934 + + + + + +-8500707880073302357 + +simple-shuffle-split + + + +-4202570559896767210->-8500707880073302357 + + + + + +-22091270831237732 + +simple-shuffle-split + + + +-4202570559896767210->-22091270831237732 + + + + + +5246200011168488393 + +simple-shuffle-split + + + +-4202570559896767210->5246200011168488393 + + + + + +4202557122767847818 + +simple-shuffle-split + + + +-4202570559896767210->4202557122767847818 + + + + + +3229557478287929959 + +simple-shuffle-split + + + +-4202570559896767210->3229557478287929959 + + + + + +-4202570559900014785 + +30 + + + +-2568180422635507275->-4202570559900014785 + + + + + +5096749064732241240 + +simple-shuffle-split + + + +-4202570559900014785->5096749064732241240 + + + + + +5395371549533020215 + +simple-shuffle-split + + + +-4202570559900014785->5395371549533020215 + + + + + +5726971721246647990 + +simple-shuffle-split + + + +-4202570559900014785->5726971721246647990 + + + + + +6853617612633974165 + +simple-shuffle-split + + + +-4202570559900014785->6853617612633974165 + + + + + +-8936168839752481676 + +simple-shuffle-split + + + +-4202570559900014785->-8936168839752481676 + + + + + +-7421663438984818301 + +simple-shuffle-split + + + +-4202570559900014785->-7421663438984818301 + + + + + +-6377181124395619326 + +simple-shuffle-split + + + +-4202570559900014785->-6377181124395619326 + + + + + +-4883471413927768351 + +simple-shuffle-split + + + +-4202570559900014785->-4883471413927768351 + + + + + +-7357211708903142560 + +simple-shuffle-split + + + +-4202570559900014785->-7357211708903142560 + + + + + +-5752113260040356385 + +simple-shuffle-split + + + +-4202570559900014785->-5752113260040356385 + + + + + +4068536220299008190 + +simple-shuffle-split + + + +-4202570559900014785->4068536220299008190 + + + + + +-3837943264414376035 + +simple-shuffle-split + + + +-4202570559900014785->-3837943264414376035 + + + + + +-2913322583994239460 + +simple-shuffle-split + + + +-4202570559900014785->-2913322583994239460 + + + + + +-1868840225116777685 + +simple-shuffle-split + + + +-4202570559900014785->-1868840225116777685 + + + + + +2446470734623623690 + +simple-shuffle-split + + + +-4202570559900014785->2446470734623623690 + + + + + +3490113550971400265 + +simple-shuffle-split + + + +-4202570559900014785->3490113550971400265 + + + + + +535520570058907656 + +simple-shuffle-split + + + +-4202570559900014785->535520570058907656 + + + + + +389755239205209831 + +simple-shuffle-split + + + +-4202570559900014785->389755239205209831 + + + + + +3050037874032122406 + +simple-shuffle-split + + + +-4202570559900014785->3050037874032122406 + + + + + +-4359083946074287035 + +simple-shuffle-split + + + +-4202570559900014785->-4359083946074287035 + + + + + +5609043529495576356 + +simple-shuffle-split + + + +-4202570559900014785->5609043529495576356 + + + + + +7103861554883143731 + +simple-shuffle-split + + + +-4202570559900014785->7103861554883143731 + + + + + +-336414278698477710 + +simple-shuffle-split + + + +-4202570559900014785->-336414278698477710 + + + + + +-7210338045358997935 + +simple-shuffle-split + + + +-4202570559900014785->-7210338045358997935 + + + + + +-2396449631314895344 + +simple-shuffle-split + + + +-4202570559900014785->-2396449631314895344 + + + + + +-828737145648533169 + +simple-shuffle-split + + + +-4202570559900014785->-828737145648533169 + + + + + +9072885264865458222 + +simple-shuffle-split + + + +-4202570559900014785->9072885264865458222 + + + + + +-8329358555274128819 + +simple-shuffle-split + + + +-4202570559900014785->-8329358555274128819 + + + + + +-4014047594459862644 + +simple-shuffle-split + + + +-4202570559900014785->-4014047594459862644 + + + + + +-2894906994614061669 + +simple-shuffle-split + + + +-4202570559900014785->-2894906994614061669 + + + + + +-1438197274491477894 + +simple-shuffle-split + + + +-4202570559900014785->-1438197274491477894 + + + + + +8379127094795577081 + +simple-shuffle-split + + + +-4202570559900014785->8379127094795577081 + + + + + +-3313947632707054728 + +simple-shuffle-split + + + +-4202570559900014785->-3313947632707054728 + + + + + +6432445041713333847 + +simple-shuffle-split + + + +-4202570559900014785->6432445041713333847 + + + + + +8596107376379563222 + +simple-shuffle-split + + + +-4202570559900014785->8596107376379563222 + + + + + +568951548004329397 + +simple-shuffle-split + + + +-4202570559900014785->568951548004329397 + + + + + +-6871359481703956844 + +simple-shuffle-split + + + +-4202570559900014785->-6871359481703956844 + + + + + +4850273718158627747 + +simple-shuffle-split + + + +-4202570559900014785->4850273718158627747 + + + + + +5893652250490188322 + +simple-shuffle-split + + + +-4202570559900014785->5893652250490188322 + + + + + +6234111350619120897 + +simple-shuffle-split + + + +-4202570559900014785->6234111350619120897 + + + + + +-6241317630063833728 + +simple-shuffle-split + + + +-4202570559900014785->-6241317630063833728 + + + + + +3735668780641649663 + +simple-shuffle-split + + + +-4202570559900014785->3735668780641649663 + + + + + +-877939447792454178 + +simple-shuffle-split + + + +-4202570559900014785->-877939447792454178 + + + + + +9091318751155150013 + +simple-shuffle-split + + + +-4202570559900014785->9091318751155150013 + + + + + +-9088965591186271428 + +simple-shuffle-split + + + +-4202570559900014785->-9088965591186271428 + + + + + +1918607630619676747 + +simple-shuffle-split + + + +-4202570559900014785->1918607630619676747 + + + + + +3596992805512103722 + +simple-shuffle-split + + + +-4202570559900014785->3596992805512103722 + + + + + +-4880515401238302103 + +simple-shuffle-split + + + +-4202570559900014785->-4880515401238302103 + + + + + +2141121775015387688 + +simple-shuffle-split + + + +-4202570559900014785->2141121775015387688 + + + + + +-4202570559898932260 + +31 + + + +-3738771995068067350->-4202570559898932260 + + + + + +-3501878531947119703 + +simple-shuffle-split + + + +-4202570559898932260->-3501878531947119703 + + + + + +-2642126062111939478 + +simple-shuffle-split + + + +-4202570559898932260->-2642126062111939478 + + + + + +-3769857899901800053 + +simple-shuffle-split + + + +-4202570559898932260->-3769857899901800053 + + + + + +-6866774455711025028 + +simple-shuffle-split + + + +-4202570559898932260->-6866774455711025028 + + + + + +-7941325392630284803 + +simple-shuffle-split + + + +-4202570559898932260->-7941325392630284803 + + + + + +-1165387318920932578 + +simple-shuffle-split + + + +-4202570559898932260->-1165387318920932578 + + + + + +7526105108686566463 + +simple-shuffle-split + + + +-4202570559898932260->7526105108686566463 + + + + + +5985139638180234688 + +simple-shuffle-split + + + +-4202570559898932260->5985139638180234688 + + + + + +8400320502111359297 + +simple-shuffle-split + + + +-4202570559898932260->8400320502111359297 + + + + + +7348051249423885922 + +simple-shuffle-split + + + +-4202570559898932260->7348051249423885922 + + + + + +6922846524790936547 + +simple-shuffle-split + + + +-4202570559898932260->6922846524790936547 + + + + + +4807936225449961172 + +simple-shuffle-split + + + +-4202570559898932260->4807936225449961172 + + + + + +1619425237804542197 + +simple-shuffle-split + + + +-4202570559898932260->1619425237804542197 + + + + + +1828354924844864022 + +simple-shuffle-split + + + +-4202570559898932260->1828354924844864022 + + + + + +8548062354837501847 + +simple-shuffle-split + + + +-4202570559898932260->8548062354837501847 + + + + + +-1201615709588672328 + +simple-shuffle-split + + + +-4202570559898932260->-1201615709588672328 + + + + + +2009830661114817113 + +simple-shuffle-split + + + +-4202570559898932260->2009830661114817113 + + + + + +-8926810899166234662 + +simple-shuffle-split + + + +-4202570559898932260->-8926810899166234662 + + + + + +8485454434646781179 + +simple-shuffle-split + + + +-4202570559898932260->8485454434646781179 + + + + + +-9102645481964936212 + +simple-shuffle-split + + + +-4202570559898932260->-9102645481964936212 + + + + + +-632887854363765587 + +simple-shuffle-split + + + +-4202570559898932260->-632887854363765587 + + + + + +5119391463924352654 + +simple-shuffle-split + + + +-4202570559898932260->5119391463924352654 + + + + + +4071588535490063279 + +simple-shuffle-split + + + +-4202570559898932260->4071588535490063279 + + + + + +-7555982554687872912 + +simple-shuffle-split + + + +-4202570559898932260->-7555982554687872912 + + + + + +4937201793546689713 + +simple-shuffle-split + + + +-4202570559898932260->4937201793546689713 + + + + + +-4947639894027167278 + +simple-shuffle-split + + + +-4202570559898932260->-4947639894027167278 + + + + + +1865375296871548563 + +simple-shuffle-split + + + +-4202570559898932260->1865375296871548563 + + + + + +813141233711667588 + +simple-shuffle-split + + + +-4202570559898932260->813141233711667588 + + + + + +462290737155496613 + +simple-shuffle-split + + + +-4202570559898932260->462290737155496613 + + + + + +7869390623887586438 + +simple-shuffle-split + + + +-4202570559898932260->7869390623887586438 + + + + + +-4915238317501041337 + +simple-shuffle-split + + + +-4202570559898932260->-4915238317501041337 + + + + + +-4627734495357634712 + +simple-shuffle-split + + + +-4202570559898932260->-4627734495357634712 + + + + + +-4501686499522562679 + +simple-shuffle-split + + + +-4202570559898932260->-4501686499522562679 + + + + + +-6132971212685552054 + +simple-shuffle-split + + + +-4202570559898932260->-6132971212685552054 + + + + + +1443807629948660587 + +simple-shuffle-split + + + +-4202570559898932260->1443807629948660587 + + + + + +8089123331206674012 + +simple-shuffle-split + + + +-4202570559898932260->8089123331206674012 + + + + + +8315788726486428637 + +simple-shuffle-split + + + +-4202570559898932260->8315788726486428637 + + + + + +7271288541713092862 + +simple-shuffle-split + + + +-4202570559898932260->7271288541713092862 + + + + + +-3031124702059794913 + +simple-shuffle-split + + + +-4202570559898932260->-3031124702059794913 + + + + + +4256133623357278112 + +simple-shuffle-split + + + +-4202570559898932260->4256133623357278112 + + + + + +4910279657982604321 + +simple-shuffle-split + + + +-4202570559898932260->4910279657982604321 + + + + + +3927873435326926146 + +simple-shuffle-split + + + +-4202570559898932260->3927873435326926146 + + + + + +2391337958911243971 + +simple-shuffle-split + + + +-4202570559898932260->2391337958911243971 + + + + + +-7346164057177584204 + +simple-shuffle-split + + + +-4202570559898932260->-7346164057177584204 + + + + + +1132451414720414421 + +simple-shuffle-split + + + +-4202570559898932260->1132451414720414421 + + + + + +643226287160365046 + +simple-shuffle-split + + + +-4202570559898932260->643226287160365046 + + + + + +-400147703623752329 + +simple-shuffle-split + + + +-4202570559898932260->-400147703623752329 + + + + + +5325544214334185112 + +simple-shuffle-split + + + +-4202570559898932260->5325544214334185112 + + + + + +-6865133564053739463 + +simple-shuffle-split + + + +-4202570559898932260->-6865133564053739463 + + + + + +-4202570559876199235 + +32 + + + +5813256369735053107->-4202570559876199235 + + + + + +482769098229217818 + +simple-shuffle-split + + + +-4202570559876199235->482769098229217818 + + + + + +2169194037029437593 + +simple-shuffle-split + + + +-4202570559876199235->2169194037029437593 + + + + + +7405038051281652268 + +simple-shuffle-split + + + +-4202570559876199235->7405038051281652268 + + + + + +-419709346855106757 + +simple-shuffle-split + + + +-4202570559876199235->-419709346855106757 + + + + + +-4509427441088549682 + +simple-shuffle-split + + + +-4202570559876199235->-4509427441088549682 + + + + + +-3601393856466732307 + +simple-shuffle-split + + + +-4202570559876199235->-3601393856466732307 + + + + + +2362553733096571568 + +simple-shuffle-split + + + +-4202570559876199235->2362553733096571568 + + + + + +-5004499729506557457 + +simple-shuffle-split + + + +-4202570559876199235->-5004499729506557457 + + + + + +-6730631202441397998 + +simple-shuffle-split + + + +-4202570559876199235->-6730631202441397998 + + + + + +3042370101369198577 + +simple-shuffle-split + + + +-4202570559876199235->3042370101369198577 + + + + + +-8184394169512837948 + +simple-shuffle-split + + + +-4202570559876199235->-8184394169512837948 + + + + + +-7065521225826569773 + +simple-shuffle-split + + + +-4202570559876199235->-7065521225826569773 + + + + + +6112037874985419718 + +simple-shuffle-split + + + +-4202570559876199235->6112037874985419718 + + + + + +-9165691997419076123 + +simple-shuffle-split + + + +-4202570559876199235->-9165691997419076123 + + + + + +-6523443078926671448 + +simple-shuffle-split + + + +-4202570559876199235->-6523443078926671448 + + + + + +5148489704822366343 + +simple-shuffle-split + + + +-4202570559876199235->5148489704822366343 + + + + + +4422422658414751402 + +simple-shuffle-split + + + +-4202570559876199235->4422422658414751402 + + + + + +5944239862976703977 + +simple-shuffle-split + + + +-4202570559876199235->5944239862976703977 + + + + + +1951109928847589052 + +simple-shuffle-split + + + +-4202570559876199235->1951109928847589052 + + + + + +2933516149268935627 + +simple-shuffle-split + + + +-4202570559876199235->2933516149268935627 + + + + + +-698881122421966498 + +simple-shuffle-split + + + +-4202570559876199235->-698881122421966498 + + + + + +1456041037964604477 + +simple-shuffle-split + + + +-4202570559876199235->1456041037964604477 + + + + + +-3678441076569365248 + +simple-shuffle-split + + + +-4202570559876199235->-3678441076569365248 + + + + + +-3886288245780332673 + +simple-shuffle-split + + + +-4202570559876199235->-3886288245780332673 + + + + + +-1728304742676626014 + +simple-shuffle-split + + + +-4202570559876199235->-1728304742676626014 + + + + + +-674962311354119039 + +simple-shuffle-split + + + +-4202570559876199235->-674962311354119039 + + + + + +-5948559388135811564 + +simple-shuffle-split + + + +-4202570559876199235->-5948559388135811564 + + + + + +-2705172533295136989 + +simple-shuffle-split + + + +-4202570559876199235->-2705172533295136989 + + + + + +8861866174736521302 + +simple-shuffle-split + + + +-4202570559876199235->8861866174736521302 + + + + + +-6837717078832731339 + +simple-shuffle-split + + + +-4202570559876199235->-6837717078832731339 + + + + + +5827429153973487352 + +simple-shuffle-split + + + +-4202570559876199235->5827429153973487352 + + + + + +8555322941840199127 + +simple-shuffle-split + + + +-4202570559876199235->8555322941840199127 + + + + + +8416094572138427386 + +simple-shuffle-split + + + +-4202570559876199235->8416094572138427386 + + + + + +1056755409582538361 + +simple-shuffle-split + + + +-4202570559876199235->1056755409582538361 + + + + + +7026031302082055436 + +simple-shuffle-split + + + +-4202570559876199235->7026031302082055436 + + + + + +7928777440997782811 + +simple-shuffle-split + + + +-4202570559876199235->7928777440997782811 + + + + + +1666220115410464686 + +simple-shuffle-split + + + +-4202570559876199235->1666220115410464686 + + + + + +2574253655692058061 + +simple-shuffle-split + + + +-4202570559876199235->2574253655692058061 + + + + + +-195684863183038064 + +simple-shuffle-split + + + +-4202570559876199235->-195684863183038064 + + + + + +-19100997865757489 + +simple-shuffle-split + + + +-4202570559876199235->-19100997865757489 + + + + + +-8996706360909327630 + +simple-shuffle-split + + + +-4202570559876199235->-8996706360909327630 + + + + + +3833057979921044945 + +simple-shuffle-split + + + +-4202570559876199235->3833057979921044945 + + + + + +-2008478922490617180 + +simple-shuffle-split + + + +-4202570559876199235->-2008478922490617180 + + + + + +-448263056214704205 + +simple-shuffle-split + + + +-4202570559876199235->-448263056214704205 + + + + + +-6158795463319703130 + +simple-shuffle-split + + + +-4202570559876199235->-6158795463319703130 + + + + + +5492764680258667461 + +simple-shuffle-split + + + +-4202570559876199235->5492764680258667461 + + + + + +-7563801960004265080 + +simple-shuffle-split + + + +-4202570559876199235->-7563801960004265080 + + + + + +-6043056782305386905 + +simple-shuffle-split + + + +-4202570559876199235->-6043056782305386905 + + + + + +-4955764415198192246 + +simple-shuffle-split + + + +-4202570559876199235->-4955764415198192246 + + + + + +-4202570559875116710 + +33 + + + +5898408974926527432->-4202570559875116710 + + + + + +2630658632484823451 + +simple-shuffle-split + + + +-4202570559875116710->2630658632484823451 + + + + + +-7220129031148890740 + +simple-shuffle-split + + + +-4202570559875116710->-7220129031148890740 + + + + + +-3319229209619479815 + +simple-shuffle-split + + + +-4202570559875116710->-3319229209619479815 + + + + + +5381122189635042426 + +simple-shuffle-split + + + +-4202570559875116710->5381122189635042426 + + + + + +-1734684433498870449 + +simple-shuffle-split + + + +-4202570559875116710->-1734684433498870449 + + + + + +5561433954438658576 + +simple-shuffle-split + + + +-4202570559875116710->5561433954438658576 + + + + + +-1541077757860184499 + +simple-shuffle-split + + + +-4202570559875116710->-1541077757860184499 + + + + + +-2520046361722321874 + +simple-shuffle-split + + + +-4202570559875116710->-2520046361722321874 + + + + + +-5772954098553506765 + +simple-shuffle-split + + + +-4202570559875116710->-5772954098553506765 + + + + + +-6123809127771755740 + +simple-shuffle-split + + + +-4202570559875116710->-6123809127771755740 + + + + + +-316626859096925615 + +simple-shuffle-split + + + +-4202570559875116710->-316626859096925615 + + + + + +-3534902409588968590 + +simple-shuffle-split + + + +-4202570559875116710->-3534902409588968590 + + + + + +8198857650907128551 + +simple-shuffle-split + + + +-4202570559875116710->8198857650907128551 + + + + + +6657758362747911176 + +simple-shuffle-split + + + +-4202570559875116710->6657758362747911176 + + + + + +2345915623735689285 + +simple-shuffle-split + + + +-4202570559875116710->2345915623735689285 + + + + + +-9139017141363440090 + +simple-shuffle-split + + + +-4202570559875116710->-9139017141363440090 + + + + + +4993832294432718635 + +simple-shuffle-split + + + +-4202570559875116710->4993832294432718635 + + + + + +3940495538676204060 + +simple-shuffle-split + + + +-4202570559875116710->3940495538676204060 + + + + + +9213252010927885385 + +simple-shuffle-split + + + +-4202570559875116710->9213252010927885385 + + + + + +8169572800124249610 + +simple-shuffle-split + + + +-4202570559875116710->8169572800124249610 + + + + + +518904536118368735 + +simple-shuffle-split + + + +-4202570559875116710->518904536118368735 + + + + + +-534437884153723040 + +simple-shuffle-split + + + +-4202570559875116710->-534437884153723040 + + + + + +4651466916419881885 + +simple-shuffle-split + + + +-4202570559875116710->4651466916419881885 + + + + + +3615821094161925310 + +simple-shuffle-split + + + +-4202570559875116710->3615821094161925310 + + + + + +-712197553789702781 + +simple-shuffle-split + + + +-4202570559875116710->-712197553789702781 + + + + + +-2678440043729743756 + +simple-shuffle-split + + + +-4202570559875116710->-2678440043729743756 + + + + + +690885871977081569 + +simple-shuffle-split + + + +-4202570559875116710->690885871977081569 + + + + + +332284831743854594 + +simple-shuffle-split + + + +-4202570559875116710->332284831743854594 + + + + + +-6307418160952923081 + +simple-shuffle-split + + + +-4202570559875116710->-6307418160952923081 + + + + + +-6530675580322969256 + +simple-shuffle-split + + + +-4202570559875116710->-6530675580322969256 + + + + + +-3714063498561703531 + +simple-shuffle-split + + + +-4202570559875116710->-3714063498561703531 + + + + + +-4138218592509439306 + +simple-shuffle-split + + + +-4202570559875116710->-4138218592509439306 + + + + + +4786485202701821883 + +simple-shuffle-split + + + +-4202570559875116710->4786485202701821883 + + + + + +3329739152434024108 + +simple-shuffle-split + + + +-4202570559875116710->3329739152434024108 + + + + + +6755741194727934233 + +simple-shuffle-split + + + +-4202570559875116710->6755741194727934233 + + + + + +-3691023047245661542 + +simple-shuffle-split + + + +-4202570559875116710->-3691023047245661542 + + + + + +-694053293746492817 + +simple-shuffle-split + + + +-4202570559875116710->-694053293746492817 + + + + + +-1767459619860573392 + +simple-shuffle-split + + + +-4202570559875116710->-1767459619860573392 + + + + + +2285260207771130733 + +simple-shuffle-split + + + +-4202570559875116710->2285260207771130733 + + + + + +-8733507365448062642 + +simple-shuffle-split + + + +-4202570559875116710->-8733507365448062642 + + + + + +5965753879828544083 + +simple-shuffle-split + + + +-4202570559875116710->5965753879828544083 + + + + + +6112660531390839108 + +simple-shuffle-split + + + +-4202570559875116710->6112660531390839108 + + + + + +136539045823458417 + +simple-shuffle-split + + + +-4202570559875116710->136539045823458417 + + + + + +-911265016612059758 + +simple-shuffle-split + + + +-4202570559875116710->-911265016612059758 + + + + + +-9097669494407351033 + +simple-shuffle-split + + + +-4202570559875116710->-9097669494407351033 + + + + + +8999332439806279208 + +simple-shuffle-split + + + +-4202570559875116710->8999332439806279208 + + + + + +-3720140440859318683 + +simple-shuffle-split + + + +-4202570559875116710->-3720140440859318683 + + + + + +752849344684607558 + +simple-shuffle-split + + + +-4202570559875116710->752849344684607558 + + + + + +8671389112616533067 + +simple-shuffle-split + + + +-4202570559875116710->8671389112616533067 + + + + + +-4202570559878364285 + +34 + + + +-8291772548664987959->-4202570559878364285 + + + + + +6683614170775187436 + +simple-shuffle-split + + + +-4202570559878364285->6683614170775187436 + + + + + +7758124251451586811 + +simple-shuffle-split + + + +-4202570559878364285->7758124251451586811 + + + + + +-9218585940201761830 + +simple-shuffle-split + + + +-4202570559878364285->-9218585940201761830 + + + + + +598742968415725145 + +simple-shuffle-split + + + +-4202570559878364285->598742968415725145 + + + + + +8344565182692097136 + +simple-shuffle-split + + + +-4202570559878364285->8344565182692097136 + + + + + +-8644324518070401105 + +simple-shuffle-split + + + +-4202570559878364285->-8644324518070401105 + + + + + +1323802984103596686 + +simple-shuffle-split + + + +-4202570559878364285->1323802984103596686 + + + + + +2306225949697138861 + +simple-shuffle-split + + + +-4202570559878364285->2306225949697138861 + + + + + +-2202114950690208380 + +simple-shuffle-split + + + +-4202570559878364285->-2202114950690208380 + + + + + +-641899111018429805 + +simple-shuffle-split + + + +-4202570559878364285->-641899111018429805 + + + + + +-8152037945810700846 + +simple-shuffle-split + + + +-4202570559878364285->-8152037945810700846 + + + + + +3529568452602452145 + +simple-shuffle-split + + + +-4202570559878364285->3529568452602452145 + + + + + +-7319257512095280280 + +simple-shuffle-split + + + +-4202570559878364285->-7319257512095280280 + + + + + +-7465860114133368505 + +simple-shuffle-split + + + +-4202570559878364285->-7465860114133368505 + + + + + +-4669016640590577530 + +simple-shuffle-split + + + +-4202570559878364285->-4669016640590577530 + + + + + +6337473968175777445 + +simple-shuffle-split + + + +-4202570559878364285->6337473968175777445 + + + + + +7784343723896759420 + +simple-shuffle-split + + + +-4202570559878364285->7784343723896759420 + + + + + +8761212649573043595 + +simple-shuffle-split + + + +-4202570559878364285->8761212649573043595 + + + + + +-5298181573109194646 + +simple-shuffle-split + + + +-4202570559878364285->-5298181573109194646 + + + + + +-4325742645974902871 + +simple-shuffle-split + + + +-4202570559878364285->-4325742645974902871 + + + + + +3383801941172969920 + +simple-shuffle-split + + + +-4202570559878364285->3383801941172969920 + + + + + +4913406049573445695 + +simple-shuffle-split + + + +-4202570559878364285->4913406049573445695 + + + + + +-3716046574174494946 + +simple-shuffle-split + + + +-4202570559878364285->-3716046574174494946 + + + + + +6739861364967336445 + +simple-shuffle-split + + + +-4202570559878364285->6739861364967336445 + + + + + +1717133504382587604 + +simple-shuffle-split + + + +-4202570559878364285->1717133504382587604 + + + + + +3206041209588625379 + +simple-shuffle-split + + + +-4202570559878364285->3206041209588625379 + + + + + +4253979053165625954 + +simple-shuffle-split + + + +-4202570559878364285->4253979053165625954 + + + + + +5818619245760376129 + +simple-shuffle-split + + + +-4202570559878364285->5818619245760376129 + + + + + +-2758066477151687496 + +simple-shuffle-split + + + +-4202570559878364285->-2758066477151687496 + + + + + +-2463837668472585321 + +simple-shuffle-split + + + +-4202570559878364285->-2463837668472585321 + + + + + +-1974592923654291946 + +simple-shuffle-split + + + +-4202570559878364285->-1974592923654291946 + + + + + +-931201090398677771 + +simple-shuffle-split + + + +-4202570559878364285->-931201090398677771 + + + + + +-5587214663671682612 + +simple-shuffle-split + + + +-4202570559878364285->-5587214663671682612 + + + + + +-5233043657452627237 + +simple-shuffle-split + + + +-4202570559878364285->-5233043657452627237 + + + + + +-4123061441899131462 + +simple-shuffle-split + + + +-4202570559878364285->-4123061441899131462 + + + + + +-2034764234739948487 + +simple-shuffle-split + + + +-4202570559878364285->-2034764234739948487 + + + + + +7233270181628122704 + +simple-shuffle-split + + + +-4202570559878364285->7233270181628122704 + + + + + +-8456657336398477937 + +simple-shuffle-split + + + +-4202570559878364285->-8456657336398477937 + + + + + +-8602422667252175762 + +simple-shuffle-split + + + +-4202570559878364285->-8602422667252175762 + + + + + +-7072801824816521587 + +simple-shuffle-split + + + +-4202570559878364285->-7072801824816521587 + + + + + +4485944375302847588 + +simple-shuffle-split + + + +-4202570559878364285->4485944375302847588 + + + + + +5533747301502805363 + +simple-shuffle-split + + + +-4202570559878364285->5533747301502805363 + + + + + +-1332125240756931662 + +simple-shuffle-split + + + +-4202570559878364285->-1332125240756931662 + + + + + +-8780183401053531503 + +simple-shuffle-split + + + +-4202570559878364285->-8780183401053531503 + + + + + +1236728470315663688 + +simple-shuffle-split + + + +-4202570559878364285->1236728470315663688 + + + + + +944528019964237863 + +simple-shuffle-split + + + +-4202570559878364285->944528019964237863 + + + + + +-5819501332670225562 + +simple-shuffle-split + + + +-4202570559878364285->-5819501332670225562 + + + + + +3996699622759655813 + +simple-shuffle-split + + + +-4202570559878364285->3996699622759655813 + + + + + +-2138528475209387428 + +simple-shuffle-split + + + +-4202570559878364285->-2138528475209387428 + + + + + +-4202570559877281760 + +35 + + + +-2682479808520719618->-4202570559877281760 + + + + + +-8374219548216852723 + +simple-shuffle-split + + + +-4202570559877281760->-8374219548216852723 + + + + + +-2700621368800260882 + +simple-shuffle-split + + + +-4202570559877281760->-2700621368800260882 + + + + + +-3674168660788982257 + +simple-shuffle-split + + + +-4202570559877281760->-3674168660788982257 + + + + + +3110641678944335568 + +simple-shuffle-split + + + +-4202570559877281760->3110641678944335568 + + + + + +-5697043306872132423 + +simple-shuffle-split + + + +-4202570559877281760->-5697043306872132423 + + + + + +1814203912623746618 + +simple-shuffle-split + + + +-4202570559877281760->1814203912623746618 + + + + + +2571885246635588443 + +simple-shuffle-split + + + +-4202570559877281760->2571885246635588443 + + + + + +1524081176457851468 + +simple-shuffle-split + + + +-4202570559877281760->1524081176457851468 + + + + + +-7357994326531260923 + +simple-shuffle-split + + + +-4202570559877281760->-7357994326531260923 + + + + + +1290651127753702 + +simple-shuffle-split + + + +-4202570559877281760->1290651127753702 + + + + + +5746656406321535143 + +simple-shuffle-split + + + +-4202570559877281760->5746656406321535143 + + + + + +5964498079323840968 + +simple-shuffle-split + + + +-4202570559877281760->5964498079323840968 + + + + + +-2768528627869738223 + +simple-shuffle-split + + + +-4202570559877281760->-2768528627869738223 + + + + + +5057363460798001202 + +simple-shuffle-split + + + +-4202570559877281760->5057363460798001202 + + + + + +-5950480900074754573 + +simple-shuffle-split + + + +-4202570559877281760->-5950480900074754573 + + + + + +-6852959276414411548 + +simple-shuffle-split + + + +-4202570559877281760->-6852959276414411548 + + + + + +2786367494190378077 + +simple-shuffle-split + + + +-4202570559877281760->2786367494190378077 + + + + + +-8875546579365174914 + +simple-shuffle-split + + + +-4202570559877281760->-8875546579365174914 + + + + + +-175195243260831073 + +simple-shuffle-split + + + +-4202570559877281760->-175195243260831073 + + + + + +-2985230133043911648 + +simple-shuffle-split + + + +-4202570559877281760->-2985230133043911648 + + + + + +9032403566833709577 + +simple-shuffle-split + + + +-4202570559877281760->9032403566833709577 + + + + + +7914639025673741002 + +simple-shuffle-split + + + +-4202570559877281760->7914639025673741002 + + + + + +6932232795258523627 + +simple-shuffle-split + + + +-4202570559877281760->6932232795258523627 + + + + + +3759914786213740252 + +simple-shuffle-split + + + +-4202570559877281760->3759914786213740252 + + + + + +-6239515116878932139 + +simple-shuffle-split + + + +-4202570559877281760->-6239515116878932139 + + + + + +-7211975355068024714 + +simple-shuffle-split + + + +-4202570559877281760->-7211975355068024714 + + + + + +689125325670827511 + +simple-shuffle-split + + + +-4202570559877281760->689125325670827511 + + + + + +-9056162422514143464 + +simple-shuffle-split + + + +-4202570559877281760->-9056162422514143464 + + + + + +1774676805403264161 + +simple-shuffle-split + + + +-4202570559877281760->1774676805403264161 + + + + + +-396869856229233214 + +simple-shuffle-split + + + +-4202570559877281760->-396869856229233214 + + + + + +-893045997883878589 + +simple-shuffle-split + + + +-4202570559877281760->-893045997883878589 + + + + + +-1795523281237013964 + +simple-shuffle-split + + + +-4202570559877281760->-1795523281237013964 + + + + + +-165799759354904275 + +simple-shuffle-split + + + +-4202570559877281760->-165799759354904275 + + + + + +-1589291178226014450 + +simple-shuffle-split + + + +-4202570559877281760->-1589291178226014450 + + + + + +-2632665133503311825 + +simple-shuffle-split + + + +-4202570559877281760->-2632665133503311825 + + + + + +-1875020195021194000 + +simple-shuffle-split + + + +-4202570559877281760->-1875020195021194000 + + + + + +6574893885743492825 + +simple-shuffle-split + + + +-4202570559877281760->6574893885743492825 + + + + + +5526992363460981850 + +simple-shuffle-split + + + +-4202570559877281760->5526992363460981850 + + + + + +3600068018425348475 + +simple-shuffle-split + + + +-4202570559877281760->3600068018425348475 + + + + + +2635375086752315500 + +simple-shuffle-split + + + +-4202570559877281760->2635375086752315500 + + + + + +-6245288008561155291 + +simple-shuffle-split + + + +-4202570559877281760->-6245288008561155291 + + + + + +-7289788262131119866 + +simple-shuffle-split + + + +-4202570559877281760->-7289788262131119866 + + + + + +-427846397552864825 + +simple-shuffle-split + + + +-4202570559877281760->-427846397552864825 + + + + + +8120329155616103400 + +simple-shuffle-split + + + +-4202570559877281760->8120329155616103400 + + + + + +-611593700592490191 + +simple-shuffle-split + + + +-4202570559877281760->-611593700592490191 + + + + + +-3424268336802523566 + +simple-shuffle-split + + + +-4202570559877281760->-3424268336802523566 + + + + + +5272798853073958675 + +simple-shuffle-split + + + +-4202570559877281760->5272798853073958675 + + + + + +-5263990145182737916 + +simple-shuffle-split + + + +-4202570559877281760->-5263990145182737916 + + + + + +3827871021476048509 + +simple-shuffle-split + + + +-4202570559877281760->3827871021476048509 + + + + + +-4202570559871869135 + +36 + + + +894390325600018007->-4202570559871869135 + + + + + +-3758663352665206962 + +simple-shuffle-split + + + +-4202570559871869135->-3758663352665206962 + + + + + +7882837374970216813 + +simple-shuffle-split + + + +-4202570559871869135->7882837374970216813 + + + + + +3187748094154717488 + +simple-shuffle-split + + + +-4202570559871869135->3187748094154717488 + + + + + +-4803142261890561937 + +simple-shuffle-split + + + +-4202570559871869135->-4803142261890561937 + + + + + +1949869690151030938 + +simple-shuffle-split + + + +-4202570559871869135->1949869690151030938 + + + + + +5137007951330412313 + +simple-shuffle-split + + + +-4202570559871869135->5137007951330412313 + + + + + +8927319731613346988 + +simple-shuffle-split + + + +-4202570559871869135->8927319731613346988 + + + + + +-314889215400903237 + +simple-shuffle-split + + + +-4202570559871869135->-314889215400903237 + + + + + +136953021108823622 + +simple-shuffle-split + + + +-4202570559871869135->136953021108823622 + + + + + +-8341642345753826203 + +simple-shuffle-split + + + +-4202570559871869135->-8341642345753826203 + + + + + +4929165357978600488 + +simple-shuffle-split + + + +-4202570559871869135->4929165357978600488 + + + + + +5972540483837811463 + +simple-shuffle-split + + + +-4202570559871869135->5972540483837811463 + + + + + +4699563700969823122 + +simple-shuffle-split + + + +-4202570559871869135->4699563700969823122 + + + + + +-3897769963834748303 + +simple-shuffle-split + + + +-4202570559871869135->-3897769963834748303 + + + + + +-9117036978490532028 + +simple-shuffle-split + + + +-4202570559871869135->-9117036978490532028 + + + + + +-8069232908312795053 + +simple-shuffle-split + + + +-4202570559871869135->-8069232908312795053 + + + + + +1315417425045856222 + +simple-shuffle-split + + + +-4202570559871869135->1315417425045856222 + + + + + +1731785579752171197 + +simple-shuffle-split + + + +-4202570559871869135->1731785579752171197 + + + + + +-1150625843348633728 + +simple-shuffle-split + + + +-4202570559871869135->-1150625843348633728 + + + + + +8603517437968270847 + +simple-shuffle-split + + + +-4202570559871869135->8603517437968270847 + + + + + +5869170301590094122 + +simple-shuffle-split + + + +-4202570559871869135->5869170301590094122 + + + + + +6145630213302817897 + +simple-shuffle-split + + + +-4202570559871869135->6145630213302817897 + + + + + +2779624831732602172 + +simple-shuffle-split + + + +-4202570559871869135->2779624831732602172 + + + + + +4708742265902729803 + +simple-shuffle-split + + + +-4202570559871869135->4708742265902729803 + + + + + +1832782470767640278 + +simple-shuffle-split + + + +-4202570559871869135->1832782470767640278 + + + + + +2877282724268323253 + +simple-shuffle-split + + + +-4202570559871869135->2877282724268323253 + + + + + +8409280786164305272 + +simple-shuffle-split + + + +-4202570559871869135->8409280786164305272 + + + + + +2041733466576470103 + +simple-shuffle-split + + + +-4202570559871869135->2041733466576470103 + + + + + +6303284766562233378 + +simple-shuffle-split + + + +-4202570559871869135->6303284766562233378 + + + + + +7356627187908189953 + +simple-shuffle-split + + + +-4202570559871869135->7356627187908189953 + + + + + +5978669010785761428 + +simple-shuffle-split + + + +-4202570559871869135->5978669010785761428 + + + + + +7014257270547600803 + +simple-shuffle-split + + + +-4202570559871869135->7014257270547600803 + + + + + +-4172619314115689938 + +simple-shuffle-split + + + +-4202570559871869135->-4172619314115689938 + + + + + +-4013730910455741363 + +simple-shuffle-split + + + +-4202570559871869135->-4013730910455741363 + + + + + +-7918773461578031088 + +simple-shuffle-split + + + +-4202570559871869135->-7918773461578031088 + + + + + +-6993044485134734513 + +simple-shuffle-split + + + +-4202570559871869135->-6993044485134734513 + + + + + +-315649725074204038 + +simple-shuffle-split + + + +-4202570559871869135->-315649725074204038 + + + + + +1647271122344870137 + +simple-shuffle-split + + + +-4202570559871869135->1647271122344870137 + + + + + +-1775678095249513588 + +simple-shuffle-split + + + +-4202570559871869135->-1775678095249513588 + + + + + +-1350164750793050213 + +simple-shuffle-split + + + +-4202570559871869135->-1350164750793050213 + + + + + +5239655478749389862 + +simple-shuffle-split + + + +-4202570559871869135->5239655478749389862 + + + + + +7237634786726061637 + +simple-shuffle-split + + + +-4202570559871869135->7237634786726061637 + + + + + +2699504060371567112 + +simple-shuffle-split + + + +-4202570559871869135->2699504060371567112 + + + + + +4240603372917907687 + +simple-shuffle-split + + + +-4202570559871869135->4240603372917907687 + + + + + +-6007555571052570254 + +simple-shuffle-split + + + +-4202570559871869135->-6007555571052570254 + + + + + +-4971932151692789679 + +simple-shuffle-split + + + +-4202570559871869135->-4971932151692789679 + + + + + +8214503783364024612 + +simple-shuffle-split + + + +-4202570559871869135->8214503783364024612 + + + + + +-9104476696104558029 + +simple-shuffle-split + + + +-4202570559871869135->-9104476696104558029 + + + + + +-1532644585536453954 + +simple-shuffle-split + + + +-4202570559871869135->-1532644585536453954 + + + + + +-4202570559870786610 + +37 + + + +-118580761502603268->-4202570559870786610 + + + + + +-1446940585988295729 + +simple-shuffle-split + + + +-4202570559870786610->-1446940585988295729 + + + + + +-1291207986217345904 + +simple-shuffle-split + + + +-4202570559870786610->-1291207986217345904 + + + + + +2852155922838332621 + +simple-shuffle-split + + + +-4202570559870786610->2852155922838332621 + + + + + +-8163192485919267154 + +simple-shuffle-split + + + +-4202570559870786610->-8163192485919267154 + + + + + +-4135808365496109029 + +simple-shuffle-split + + + +-4202570559870786610->-4135808365496109029 + + + + + +3825666883521373196 + +simple-shuffle-split + + + +-4202570559870786610->3825666883521373196 + + + + + +-38731361119123079 + +simple-shuffle-split + + + +-4202570559870786610->-38731361119123079 + + + + + +6751911098090385146 + +simple-shuffle-split + + + +-4202570559870786610->6751911098090385146 + + + + + +1493888516157453671 + +simple-shuffle-split + + + +-4202570559870786610->1493888516157453671 + + + + + +-176746115057917304 + +simple-shuffle-split + + + +-4202570559870786610->-176746115057917304 + + + + + +-5678800229175163195 + +simple-shuffle-split + + + +-4202570559870786610->-5678800229175163195 + + + + + +-7219921987920679770 + +simple-shuffle-split + + + +-4202570559870786610->-7219921987920679770 + + + + + +5276162369028680371 + +simple-shuffle-split + + + +-4202570559870786610->5276162369028680371 + + + + + +4293756102050098596 + +simple-shuffle-split + + + +-4202570559870786610->4293756102050098596 + + + + + +-7718683402909102895 + +simple-shuffle-split + + + +-4202570559870786610->-7718683402909102895 + + + + + +-8793198059513939470 + +simple-shuffle-split + + + +-4202570559870786610->-8793198059513939470 + + + + + +3647494264493916255 + +simple-shuffle-split + + + +-4202570559870786610->3647494264493916255 + + + + + +-6253082962413285920 + +simple-shuffle-split + + + +-4202570559870786610->-6253082962413285920 + + + + + +-4434004478321012195 + +simple-shuffle-split + + + +-4202570559870786610->-4434004478321012195 + + + + + +4052381196600858430 + +simple-shuffle-split + + + +-4202570559870786610->4052381196600858430 + + + + + +-9138497561212632661 + +simple-shuffle-split + + + +-4202570559870786610->-9138497561212632661 + + + + + +-1699331213167282020 + +simple-shuffle-split + + + +-4202570559870786610->-1699331213167282020 + + + + + +2324711089811746505 + +simple-shuffle-split + + + +-4202570559870786610->2324711089811746505 + + + + + +767553786986171530 + +simple-shuffle-split + + + +-4202570559870786610->767553786986171530 + + + + + +6496178653449948855 + +simple-shuffle-split + + + +-4202570559870786610->6496178653449948855 + + + + + +-5234313519220125736 + +simple-shuffle-split + + + +-4202570559870786610->-5234313519220125736 + + + + + +7880034129525085205 + +simple-shuffle-split + + + +-4202570559870786610->7880034129525085205 + + + + + +6358163891803576630 + +simple-shuffle-split + + + +-4202570559870786610->6358163891803576630 + + + + + +-8735842498284990461 + +simple-shuffle-split + + + +-4202570559870786610->-8735842498284990461 + + + + + +444188359904606964 + +simple-shuffle-split + + + +-4202570559870786610->444188359904606964 + + + + + +-5519813459986980511 + +simple-shuffle-split + + + +-4202570559870786610->-5519813459986980511 + + + + + +-7692468506392367486 + +simple-shuffle-split + + + +-4202570559870786610->-7692468506392367486 + + + + + +642441079958769903 + +simple-shuffle-split + + + +-4202570559870786610->642441079958769903 + + + + + +-943408806481454672 + +simple-shuffle-split + + + +-4202570559870786610->-943408806481454672 + + + + + +4792756547754435053 + +simple-shuffle-split + + + +-4202570559870786610->4792756547754435053 + + + + + +2128027107137020878 + +simple-shuffle-split + + + +-4202570559870786610->2128027107137020878 + + + + + +6320910032015839803 + +simple-shuffle-split + + + +-4202570559870786610->6320910032015839803 + + + + + +5837240041724310828 + +simple-shuffle-split + + + +-4202570559870786610->5837240041724310828 + + + + + +-9109152310956175463 + +simple-shuffle-split + + + +-4202570559870786610->-9109152310956175463 + + + + + +6607389439887159578 + +simple-shuffle-split + + + +-4202570559870786610->6607389439887159578 + + + + + +-8273619780665787513 + +simple-shuffle-split + + + +-4202570559870786610->-8273619780665787513 + + + + + +9200721691603343528 + +simple-shuffle-split + + + +-4202570559870786610->9200721691603343528 + + + + + +7250192934157287653 + +simple-shuffle-split + + + +-4202570559870786610->7250192934157287653 + + + + + +6206796497208635078 + +simple-shuffle-split + + + +-4202570559870786610->6206796497208635078 + + + + + +9173164231443771603 + +simple-shuffle-split + + + +-4202570559870786610->9173164231443771603 + + + + + +6388408506220249028 + +simple-shuffle-split + + + +-4202570559870786610->6388408506220249028 + + + + + +-7907460042387336463 + +simple-shuffle-split + + + +-4202570559870786610->-7907460042387336463 + + + + + +-7751730854631264238 + +simple-shuffle-split + + + +-4202570559870786610->-7751730854631264238 + + + + + +-4873060480262858113 + +simple-shuffle-split + + + +-4202570559870786610->-4873060480262858113 + + + + + +-4202570559874034185 + +38 + + + +4797800327400273357->-4202570559874034185 + + + + + +-2275926841802611984 + +simple-shuffle-split + + + +-4202570559874034185->-2275926841802611984 + + + + + +-2501523671008416209 + +simple-shuffle-split + + + +-4202570559874034185->-2501523671008416209 + + + + + +685614591244829966 + +simple-shuffle-split + + + +-4202570559874034185->685614591244829966 + + + + + +2858287509046781741 + +simple-shuffle-split + + + +-4202570559874034185->2858287509046781741 + + + + + +-6238596950200801684 + +simple-shuffle-split + + + +-4202570559874034185->-6238596950200801684 + + + + + +-5265049701461119109 + +simple-shuffle-split + + + +-4202570559874034185->-5265049701461119109 + + + + + +6394147626301528666 + +simple-shuffle-split + + + +-4202570559874034185->6394147626301528666 + + + + + +7364236061603278041 + +simple-shuffle-split + + + +-4202570559874034185->7364236061603278041 + + + + + +7155285064633981416 + +simple-shuffle-split + + + +-4202570559874034185->7155285064633981416 + + + + + +510240498043393991 + +simple-shuffle-split + + + +-4202570559874034185->510240498043393991 + + + + + +-8200114494783027450 + +simple-shuffle-split + + + +-4202570559874034185->-8200114494783027450 + + + + + +3471799864563714341 + +simple-shuffle-split + + + +-4202570559874034185->3471799864563714341 + + + + + +4448668800233869316 + +simple-shuffle-split + + + +-4202570559874034185->4448668800233869316 + + + + + +-1223838892427852525 + +simple-shuffle-split + + + +-4202570559874034185->-1223838892427852525 + + + + + +8673353518418320466 + +simple-shuffle-split + + + +-4202570559874034185->8673353518418320466 + + + + + +7841281703797198641 + +simple-shuffle-split + + + +-4202570559874034185->7841281703797198641 + + + + + +-8371835006123505600 + +simple-shuffle-split + + + +-4202570559874034185->-8371835006123505600 + + + + + +1457802644476854975 + +simple-shuffle-split + + + +-4202570559874034185->1457802644476854975 + + + + + +2994343788647029150 + +simple-shuffle-split + + + +-4202570559874034185->2994343788647029150 + + + + + +-5562978043950412675 + +simple-shuffle-split + + + +-4202570559874034185->-5562978043950412675 + + + + + +8340829643911592700 + +simple-shuffle-split + + + +-4202570559874034185->8340829643911592700 + + + + + +-207613661874215925 + +simple-shuffle-split + + + +-4202570559874034185->-207613661874215925 + + + + + +296313021205833450 + +simple-shuffle-split + + + +-4202570559874034185->296313021205833450 + + + + + +2385718588681501225 + +simple-shuffle-split + + + +-4202570559874034185->2385718588681501225 + + + + + +-5592076947777263816 + +simple-shuffle-split + + + +-4202570559874034185->-5592076947777263816 + + + + + +4377155478415140375 + +simple-shuffle-split + + + +-4202570559874034185->4377155478415140375 + + + + + +4093817371854054550 + +simple-shuffle-split + + + +-4202570559874034185->4093817371854054550 + + + + + +6970508961356401525 + +simple-shuffle-split + + + +-4202570559874034185->6970508961356401525 + + + + + +-536004181321414316 + +simple-shuffle-split + + + +-4202570559874034185->-536004181321414316 + + + + + +-8511067914047203741 + +simple-shuffle-split + + + +-4202570559874034185->-8511067914047203741 + + + + + +-7537556918931207966 + +simple-shuffle-split + + + +-4202570559874034185->-7537556918931207966 + + + + + +-4360808935731555391 + +simple-shuffle-split + + + +-4202570559874034185->-4360808935731555391 + + + + + +-3387261616047514416 + +simple-shuffle-split + + + +-4202570559874034185->-3387261616047514416 + + + + + +-2955797899733881841 + +simple-shuffle-split + + + +-4202570559874034185->-2955797899733881841 + + + + + +-1911315585179323666 + +simple-shuffle-split + + + +-4202570559874034185->-1911315585179323666 + + + + + +7838415564540810509 + +simple-shuffle-split + + + +-4202570559874034185->7838415564540810509 + + + + + +-62681710938120116 + +simple-shuffle-split + + + +-4202570559874034185->-62681710938120116 + + + + + +909757216213492059 + +simple-shuffle-split + + + +-4202570559874034185->909757216213492059 + + + + + +5353749060723455034 + +simple-shuffle-split + + + +-4202570559874034185->5353749060723455034 + + + + + +6251800839485582009 + +simple-shuffle-split + + + +-4202570559874034185->6251800839485582009 + + + + + +6756868809984419784 + +simple-shuffle-split + + + +-4202570559874034185->6756868809984419784 + + + + + +8919524777830497959 + +simple-shuffle-split + + + +-4202570559874034185->8919524777830497959 + + + + + +-8653548083575584282 + +simple-shuffle-split + + + +-4202570559874034185->-8653548083575584282 + + + + + +-7605727356906099707 + +simple-shuffle-split + + + +-4202570559874034185->-7605727356906099707 + + + + + +-7822428860954447132 + +simple-shuffle-split + + + +-4202570559874034185->-7822428860954447132 + + + + + +-5012393971223327757 + +simple-shuffle-split + + + +-4202570559874034185->-5012393971223327757 + + + + + +-3597475316028549582 + +simple-shuffle-split + + + +-4202570559874034185->-3597475316028549582 + + + + + +-2124938975929313007 + +simple-shuffle-split + + + +-4202570559874034185->-2124938975929313007 + + + + + +38857142008807968 + +simple-shuffle-split + + + +-4202570559874034185->38857142008807968 + + + + + +-4202570559872951660 + +39 + + + +-7526970341236815518->-4202570559872951660 + + + + + +-7272794668948052927 + +simple-shuffle-split + + + +-4202570559872951660->-7272794668948052927 + + + + + +-1545146796184432286 + +simple-shuffle-split + + + +-4202570559872951660->-1545146796184432286 + + + + + +5814156086091674339 + +simple-shuffle-split + + + +-4202570559872951660->5814156086091674339 + + + + + +5436733318037790164 + +simple-shuffle-split + + + +-4202570559872951660->5436733318037790164 + + + + + +4388625239350553589 + +simple-shuffle-split + + + +-4202570559872951660->4388625239350553589 + + + + + +-7245085214589322986 + +simple-shuffle-split + + + +-4202570559872951660->-7245085214589322986 + + + + + +1224672456174284439 + +simple-shuffle-split + + + +-4202570559872951660->1224672456174284439 + + + + + +-242040856342411336 + +simple-shuffle-split + + + +-4202570559872951660->-242040856342411336 + + + + + +-1276555873175892311 + +simple-shuffle-split + + + +-4202570559872951660->-1276555873175892311 + + + + + +6233582989329018730 + +simple-shuffle-split + + + +-4202570559872951660->6233582989329018730 + + + + + +-4255816353658239861 + +simple-shuffle-split + + + +-4202570559872951660->-4255816353658239861 + + + + + +-5303754197183279236 + +simple-shuffle-split + + + +-4202570559872951660->-5303754197183279236 + + + + + +-8476072205171518211 + +simple-shuffle-split + + + +-4202570559872951660->-8476072205171518211 + + + + + +8896140484583692830 + +simple-shuffle-split + + + +-4202570559872951660->8896140484583692830 + + + + + +-8690851117108308161 + +simple-shuffle-split + + + +-4202570559872951660->-8690851117108308161 + + + + + +7512306250801369280 + +simple-shuffle-split + + + +-4202570559872951660->7512306250801369280 + + + + + +5456817685685366705 + +simple-shuffle-split + + + +-4202570559872951660->5456817685685366705 + + + + + +5611478323925790930 + +simple-shuffle-split + + + +-4202570559872951660->5611478323925790930 + + + + + +4639039389031959955 + +simple-shuffle-split + + + +-4202570559872951660->4639039389031959955 + + + + + +251685005342568580 + +simple-shuffle-split + + + +-4202570559872951660->251685005342568580 + + + + + +-725182865053704795 + +simple-shuffle-split + + + +-4202570559872951660->-725182865053704795 + + + + + +-1146986253355266170 + +simple-shuffle-split + + + +-4202570559872951660->-1146986253355266170 + + + + + +-3261896589259605945 + +simple-shuffle-split + + + +-4202570559872951660->-3261896589259605945 + + + + + +-4108119666275828120 + +simple-shuffle-split + + + +-4202570559872951660->-4108119666275828120 + + + + + +-5235847009058160295 + +simple-shuffle-split + + + +-4202570559872951660->-5235847009058160295 + + + + + +-6701451917992012070 + +simple-shuffle-split + + + +-4202570559872951660->-6701451917992012070 + + + + + +-7737075337351792645 + +simple-shuffle-split + + + +-4202570559872951660->-7737075337351792645 + + + + + +-9155011655465785620 + +simple-shuffle-split + + + +-4202570559872951660->-9155011655465785620 + + + + + +7734801968595093421 + +simple-shuffle-split + + + +-4202570559872951660->7734801968595093421 + + + + + +6753487371146856846 + +simple-shuffle-split + + + +-4202570559872951660->6753487371146856846 + + + + + +3435745160896529071 + +simple-shuffle-split + + + +-4202570559872951660->3435745160896529071 + + + + + +2511083606618685296 + +simple-shuffle-split + + + +-4202570559872951660->2511083606618685296 + + + + + +7682836455031130913 + +simple-shuffle-split + + + +-4202570559872951660->7682836455031130913 + + + + + +-3809861894988259262 + +simple-shuffle-split + + + +-4202570559872951660->-3809861894988259262 + + + + + +-4854345387918911037 + +simple-shuffle-split + + + +-4202570559872951660->-4854345387918911037 + + + + + +-5269582772672929612 + +simple-shuffle-split + + + +-4202570559872951660->-5269582772672929612 + + + + + +2017676642439789013 + +simple-shuffle-split + + + +-4202570559872951660->2017676642439789013 + + + + + +-7732022688666321162 + +simple-shuffle-split + + + +-4202570559872951660->-7732022688666321162 + + + + + +7958687688092380279 + +simple-shuffle-split + + + +-4202570559872951660->7958687688092380279 + + + + + +-1938509263227371112 + +simple-shuffle-split + + + +-4202570559872951660->-1938509263227371112 + + + + + +3787220123345672329 + +simple-shuffle-split + + + +-4202570559872951660->3787220123345672329 + + + + + +2743844989761562954 + +simple-shuffle-split + + + +-4202570559872951660->2743844989761562954 + + + + + +3519239621389763179 + +simple-shuffle-split + + + +-4202570559872951660->3519239621389763179 + + + + + +-6448887844186229412 + +simple-shuffle-split + + + +-4202570559872951660->-6448887844186229412 + + + + + +-613300310582164771 + +simple-shuffle-split + + + +-4202570559872951660->-613300310582164771 + + + + + +-1661125649674169346 + +simple-shuffle-split + + + +-4202570559872951660->-1661125649674169346 + + + + + +-1444424117913181921 + +simple-shuffle-split + + + +-4202570559872951660->-1444424117913181921 + + + + + +-4640417911081350496 + +simple-shuffle-split + + + +-4202570559872951660->-4640417911081350496 + + + + + +-6804214030127977071 + +simple-shuffle-split + + + +-4202570559872951660->-6804214030127977071 + + + + + +-4202570559884859435 + +40 + + + +8816635300898960907->-4202570559884859435 + + + + + +-354591326279656430 + +simple-shuffle-split + + + +-4202570559884859435->-354591326279656430 + + + + + +1185367853893410545 + +simple-shuffle-split + + + +-4202570559884859435->1185367853893410545 + + + + + +6085564958226103236 + +simple-shuffle-split + + + +-4202570559884859435->6085564958226103236 + + + + + +-1429137792526549805 + +simple-shuffle-split + + + +-4202570559884859435->-1429137792526549805 + + + + + +-5451825285884835130 + +simple-shuffle-split + + + +-4202570559884859435->-5451825285884835130 + + + + + +-4408433460319478555 + +simple-shuffle-split + + + +-4202570559884859435->-4408433460319478555 + + + + + +8857944242671264936 + +simple-shuffle-split + + + +-4202570559884859435->8857944242671264936 + + + + + +-8540995840655632505 + +simple-shuffle-split + + + +-4202570559884859435->-8540995840655632505 + + + + + +7325330038691708186 + +simple-shuffle-split + + + +-4202570559884859435->7325330038691708186 + + + + + +-8878932280949015655 + +simple-shuffle-split + + + +-4202570559884859435->-8878932280949015655 + + + + + +5534360125878440236 + +simple-shuffle-split + + + +-4202570559884859435->5534360125878440236 + + + + + +5417549409115684411 + +simple-shuffle-split + + + +-4202570559884859435->5417549409115684411 + + + + + +174585261609844686 + +simple-shuffle-split + + + +-4202570559884859435->174585261609844686 + + + + + +4489876640776609261 + +simple-shuffle-split + + + +-4202570559884859435->4489876640776609261 + + + + + +-1224069255223990864 + +simple-shuffle-split + + + +-4202570559884859435->-1224069255223990864 + + + + + +-869897051836207889 + +simple-shuffle-split + + + +-4202570559884859435->-869897051836207889 + + + + + +-4784801422360516446 + +simple-shuffle-split + + + +-4202570559884859435->-4784801422360516446 + + + + + +5031381662781305729 + +simple-shuffle-split + + + +-4202570559884859435->5031381662781305729 + + + + + +1008747192606026004 + +simple-shuffle-split + + + +-4202570559884859435->1008747192606026004 + + + + + +2052089408025158179 + +simple-shuffle-split + + + +-4202570559884859435->2052089408025158179 + + + + + +-3123815178827442346 + +simple-shuffle-split + + + +-4202570559884859435->-3123815178827442346 + + + + + +6844298409384264245 + +simple-shuffle-split + + + +-4202570559884859435->6844298409384264245 + + + + + +-4586504018197119496 + +simple-shuffle-split + + + +-4202570559884859435->-4586504018197119496 + + + + + +-2422806564999617321 + +simple-shuffle-split + + + +-4202570559884859435->-2422806564999617321 + + + + + +2547315866271084970 + +simple-shuffle-split + + + +-4202570559884859435->2547315866271084970 + + + + + +2281687721763725545 + +simple-shuffle-split + + + +-4202570559884859435->2281687721763725545 + + + + + +-2812478587456512580 + +simple-shuffle-split + + + +-4202570559884859435->-2812478587456512580 + + + + + +-7421921102577268021 + +simple-shuffle-split + + + +-4202570559884859435->-7421921102577268021 + + + + + +5249734669757464670 + +simple-shuffle-split + + + +-4202570559884859435->5249734669757464670 + + + + + +5665029654696790845 + +simple-shuffle-split + + + +-4202570559884859435->5665029654696790845 + + + + + +-7283245134133396480 + +simple-shuffle-split + + + +-4202570559884859435->-7283245134133396480 + + + + + +4204983458625607295 + +simple-shuffle-split + + + +-4202570559884859435->4204983458625607295 + + + + + +7468162837926282738 + +simple-shuffle-split + + + +-4202570559884859435->7468162837926282738 + + + + + +-1009345412021200687 + +simple-shuffle-split + + + +-4202570559884859435->-1009345412021200687 + + + + + +-6260762785943160412 + +simple-shuffle-split + + + +-4202570559884859435->-6260762785943160412 + + + + + +-5216280435933743437 + +simple-shuffle-split + + + +-4202570559884859435->-5216280435933743437 + + + + + +3067621092943644838 + +simple-shuffle-split + + + +-4202570559884859435->3067621092943644838 + + + + + +-5409869252352105787 + +simple-shuffle-split + + + +-4202570559884859435->-5409869252352105787 + + + + + +7820907074002452104 + +simple-shuffle-split + + + +-4202570559884859435->7820907074002452104 + + + + + +8794454331627499879 + +simple-shuffle-split + + + +-4202570559884859435->8794454331627499879 + + + + + +4837185910399957754 + +simple-shuffle-split + + + +-4202570559884859435->4837185910399957754 + + + + + +7963351837550974329 + +simple-shuffle-split + + + +-4202570559884859435->7963351837550974329 + + + + + +-6736736392440602612 + +simple-shuffle-split + + + +-4202570559884859435->-6736736392440602612 + + + + + +2603558518881149979 + +simple-shuffle-split + + + +-4202570559884859435->2603558518881149979 + + + + + +-827154606533803346 + +simple-shuffle-split + + + +-4202570559884859435->-827154606533803346 + + + + + +-7781219920774151987 + +simple-shuffle-split + + + +-4202570559884859435->-7781219920774151987 + + + + + +6105928136805058704 + +simple-shuffle-split + + + +-4202570559884859435->6105928136805058704 + + + + + +-1946028650983452721 + +simple-shuffle-split + + + +-4202570559884859435->-1946028650983452721 + + + + + +199755535346145922 + +simple-shuffle-split + + + +-4202570559884859435->199755535346145922 + + + + + +-4202570559883776910 + +41 + + + +6454654824373900032->-4202570559883776910 + + + + + +1770010154810922803 + +simple-shuffle-split + + + +-4202570559883776910->1770010154810922803 + + + + + +-9165486688265732572 + +simple-shuffle-split + + + +-4202570559883776910->-9165486688265732572 + + + + + +3245989175914275153 + +simple-shuffle-split + + + +-4202570559883776910->3245989175914275153 + + + + + +3392862873804772978 + +simple-shuffle-split + + + +-4202570559883776910->3392862873804772978 + + + + + +-5490628949868201497 + +simple-shuffle-split + + + +-4202570559883776910->-5490628949868201497 + + + + + +-6467496822481486072 + +simple-shuffle-split + + + +-4202570559883776910->-6467496822481486072 + + + + + +-74176913691740347 + +simple-shuffle-split + + + +-4202570559883776910->-74176913691740347 + + + + + +-1046600791857928922 + +simple-shuffle-split + + + +-4202570559883776910->-1046600791857928922 + + + + + +9090613057646475419 + +simple-shuffle-split + + + +-4202570559883776910->9090613057646475419 + + + + + +8658849804534933644 + +simple-shuffle-split + + + +-4202570559883776910->8658849804534933644 + + + + + +-7511980271019952647 + +simple-shuffle-split + + + +-4202570559883776910->-7511980271019952647 + + + + + +-9060830207090393222 + +simple-shuffle-split + + + +-4202570559883776910->-9060830207090393222 + + + + + +3535022293046256719 + +simple-shuffle-split + + + +-4202570559883776910->3535022293046256719 + + + + + +1974774643715100944 + +simple-shuffle-split + + + +-4202570559883776910->1974774643715100944 + + + + + +7685302546944526669 + +simple-shuffle-split + + + +-4202570559883776910->7685302546944526669 + + + + + +4949636904311317294 + +simple-shuffle-split + + + +-4202570559883776910->4949636904311317294 + + + + + +-3135611688341330813 + +simple-shuffle-split + + + +-4202570559883776910->-3135611688341330813 + + + + + +5412831580863793012 + +simple-shuffle-split + + + +-4202570559883776910->5412831580863793012 + + + + + +8230529654459002337 + +simple-shuffle-split + + + +-4202570559883776910->8230529654459002337 + + + + + +-1515826767394243838 + +simple-shuffle-split + + + +-4202570559883776910->-1515826767394243838 + + + + + +-981462841803047113 + +simple-shuffle-split + + + +-4202570559883776910->-981462841803047113 + + + + + +-2596952707011531688 + +simple-shuffle-split + + + +-4202570559883776910->-2596952707011531688 + + + + + +-8552078854458765163 + +simple-shuffle-split + + + +-4202570559883776910->-8552078854458765163 + + + + + +70770096275611062 + +simple-shuffle-split + + + +-4202570559883776910->70770096275611062 + + + + + +-4300885774567123413 + +simple-shuffle-split + + + +-4202570559883776910->-4300885774567123413 + + + + + +-7110957025498933988 + +simple-shuffle-split + + + +-4202570559883776910->-7110957025498933988 + + + + + +6029141879724942153 + +simple-shuffle-split + + + +-4202570559883776910->6029141879724942153 + + + + + +-4517623639816162038 + +simple-shuffle-split + + + +-4202570559883776910->-4517623639816162038 + + + + + +7400675883626311903 + +simple-shuffle-split + + + +-4202570559883776910->7400675883626311903 + + + + + +-3610184890931808672 + +simple-shuffle-split + + + +-4202570559883776910->-3610184890931808672 + + + + + +399181845804740253 + +simple-shuffle-split + + + +-4202570559883776910->399181845804740253 + + + + + +8448613729385682878 + +simple-shuffle-split + + + +-4202570559883776910->8448613729385682878 + + + + + +2882449918476062035 + +simple-shuffle-split + + + +-4202570559883776910->2882449918476062035 + + + + + +-6934879017802103740 + +simple-shuffle-split + + + +-4202570559883776910->-6934879017802103740 + + + + + +-2928854064163331215 + +simple-shuffle-split + + + +-4202570559883776910->-2928854064163331215 + + + + + +-4485744777246401390 + +simple-shuffle-split + + + +-4202570559883776910->-4485744777246401390 + + + + + +-1680193678458729465 + +simple-shuffle-split + + + +-4202570559883776910->-1680193678458729465 + + + + + +5803332011965383976 + +simple-shuffle-split + + + +-4202570559883776910->5803332011965383976 + + + + + +-8558119436664194715 + +simple-shuffle-split + + + +-4202570559883776910->-8558119436664194715 + + + + + +8918514935029030726 + +simple-shuffle-split + + + +-4202570559883776910->8918514935029030726 + + + + + +22860363994725051 + +simple-shuffle-split + + + +-4202570559883776910->22860363994725051 + + + + + +-941791631675005524 + +simple-shuffle-split + + + +-4202570559883776910->-941791631675005524 + + + + + +5382654817774283801 + +simple-shuffle-split + + + +-4202570559883776910->5382654817774283801 + + + + + +4400247452214691226 + +simple-shuffle-split + + + +-4202570559883776910->4400247452214691226 + + + + + +-4949662311517704849 + +simple-shuffle-split + + + +-4202570559883776910->-4949662311517704849 + + + + + +-4239758393559388624 + +simple-shuffle-split + + + +-4202570559883776910->-4239758393559388624 + + + + + +-794916797531626899 + +simple-shuffle-split + + + +-4202570559883776910->-794916797531626899 + + + + + +6061239370187931726 + +simple-shuffle-split + + + +-4202570559883776910->6061239370187931726 + + + + + +-2024317778046866781 + +simple-shuffle-split + + + +-4202570559883776910->-2024317778046866781 + + + + + +-4202570559887024485 + +42 + + + +-6433497853406125759->-4202570559887024485 + + + + + +8386346349976722564 + +simple-shuffle-split + + + +-4202570559887024485->8386346349976722564 + + + + + +-8574851450908231277 + +simple-shuffle-split + + + +-4202570559887024485->-8574851450908231277 + + + + + +-5817998570605273902 + +simple-shuffle-split + + + +-4202570559887024485->-5817998570605273902 + + + + + +-6034700103474766927 + +simple-shuffle-split + + + +-4202570559887024485->-6034700103474766927 + + + + + +3907789468740405864 + +simple-shuffle-split + + + +-4202570559887024485->3907789468740405864 + + + + + +7080375167217210439 + +simple-shuffle-split + + + +-4202570559887024485->7080375167217210439 + + + + + +8043955121561332614 + +simple-shuffle-split + + + +-4202570559887024485->8043955121561332614 + + + + + +9088458790437050789 + +simple-shuffle-split + + + +-4202570559887024485->9088458790437050789 + + + + + +-1686012982851190036 + +simple-shuffle-split + + + +-4202570559887024485->-1686012982851190036 + + + + + +-45018303782624261 + +simple-shuffle-split + + + +-4202570559887024485->-45018303782624261 + + + + + +179347482624221914 + +simple-shuffle-split + + + +-4202570559887024485->179347482624221914 + + + + + +2934278498063494489 + +simple-shuffle-split + + + +-4202570559887024485->2934278498063494489 + + + + + +-6722534813472699536 + +simple-shuffle-split + + + +-4202570559887024485->-6722534813472699536 + + + + + +-5745666950870606161 + +simple-shuffle-split + + + +-4202570559887024485->-5745666950870606161 + + + + + +-4260116181929974386 + +simple-shuffle-split + + + +-4202570559887024485->-4260116181929974386 + + + + + +-2730495331734781011 + +simple-shuffle-split + + + +-4202570559887024485->-2730495331734781011 + + + + + +-3942248176917517868 + +simple-shuffle-split + + + +-4202570559887024485->-3942248176917517868 + + + + + +6026987645718724323 + +simple-shuffle-split + + + +-4202570559887024485->6026987645718724323 + + + + + +6515124041236307298 + +simple-shuffle-split + + + +-4202570559887024485->6515124041236307298 + + + + + +-2114331995600375743 + +simple-shuffle-split + + + +-4202570559887024485->-2114331995600375743 + + + + + +6568798028934046648 + +simple-shuffle-split + + + +-4202570559887024485->6568798028934046648 + + + + + +-9130766698181113193 + +simple-shuffle-split + + + +-4202570559887024485->-9130766698181113193 + + + + + +775284604603602198 + +simple-shuffle-split + + + +-4202570559887024485->775284604603602198 + + + + + +-4906100434311288843 + +simple-shuffle-split + + + +-4202570559887024485->-4906100434311288843 + + + + + +2199316226298567548 + +simple-shuffle-split + + + +-4202570559887024485->2199316226298567548 + + + + + +3730024123555754123 + +simple-shuffle-split + + + +-4202570559887024485->3730024123555754123 + + + + + +4773402691324853098 + +simple-shuffle-split + + + +-4202570559887024485->4773402691324853098 + + + + + +-3180879990026077527 + +simple-shuffle-split + + + +-4202570559887024485->-3180879990026077527 + + + + + +-2843168091955794752 + +simple-shuffle-split + + + +-4202570559887024485->-2843168091955794752 + + + + + +-744903597069405377 + +simple-shuffle-split + + + +-4202570559887024485->-744903597069405377 + + + + + +-249799518927800802 + +simple-shuffle-split + + + +-4202570559887024485->-249799518927800802 + + + + + +-8807116853209517827 + +simple-shuffle-split + + + +-4202570559887024485->-8807116853209517827 + + + + + +7235820838286860132 + +simple-shuffle-split + + + +-4202570559887024485->7235820838286860132 + + + + + +1628845861715352691 + +simple-shuffle-split + + + +-4202570559887024485->1628845861715352691 + + + + + +-6929328770047565134 + +simple-shuffle-split + + + +-4202570559887024485->-6929328770047565134 + + + + + +-5881525904798094959 + +simple-shuffle-split + + + +-4202570559887024485->-5881525904798094959 + + + + + +-8398605553940883384 + +simple-shuffle-split + + + +-4202570559887024485->-8398605553940883384 + + + + + +3331886618729191207 + +simple-shuffle-split + + + +-4202570559887024485->3331886618729191207 + + + + + +-4227142530758939034 + +simple-shuffle-split + + + +-4202570559887024485->-4227142530758939034 + + + + + +6254522931676724357 + +simple-shuffle-split + + + +-4202570559887024485->6254522931676724357 + + + + + +-3311249421301353268 + +simple-shuffle-split + + + +-4202570559887024485->-3311249421301353268 + + + + + +-2346597353578758693 + +simple-shuffle-split + + + +-4202570559887024485->-2346597353578758693 + + + + + +8585480326162129082 + +simple-shuffle-split + + + +-4202570559887024485->8585480326162129082 + + + + + +-8256160720556508359 + +simple-shuffle-split + + + +-4202570559887024485->-8256160720556508359 + + + + + +-644298177854817968 + +simple-shuffle-split + + + +-4202570559887024485->-644298177854817968 + + + + + +430248289500581007 + +simple-shuffle-split + + + +-4202570559887024485->430248289500581007 + + + + + +3617386551753827182 + +simple-shuffle-split + + + +-4202570559887024485->3617386551753827182 + + + + + +5790059440734633357 + +simple-shuffle-split + + + +-4202570559887024485->5790059440734633357 + + + + + +2233399299769093300 + +simple-shuffle-split + + + +-4202570559887024485->2233399299769093300 + + + + + +-4202570559885941960 + +43 + + + +1926760143565546966->-4202570559885941960 + + + + + +2170820924475612421 + +simple-shuffle-split + + + +-4202570559885941960->2170820924475612421 + + + + + +1745594897655907046 + +simple-shuffle-split + + + +-4202570559885941960->1745594897655907046 + + + + + +711075339292303271 + +simple-shuffle-split + + + +-4202570559885941960->711075339292303271 + + + + + +-1251840977629742904 + +simple-shuffle-split + + + +-4202570559885941960->-1251840977629742904 + + + + + +8309773607733845521 + +simple-shuffle-split + + + +-4202570559885941960->8309773607733845521 + + + + + +7265291214475389746 + +simple-shuffle-split + + + +-4202570559885941960->7265291214475389746 + + + + + +7456220286210633971 + +simple-shuffle-split + + + +-4202570559885941960->7456220286210633971 + + + + + +3148696193676109796 + +simple-shuffle-split + + + +-4202570559885941960->3148696193676109796 + + + + + +-7819092055308974579 + +simple-shuffle-split + + + +-4202570559885941960->-7819092055308974579 + + + + + +-8872451248413940754 + +simple-shuffle-split + + + +-4202570559885941960->-8872451248413940754 + + + + + +6402985717915427087 + +simple-shuffle-split + + + +-4202570559885941960->6402985717915427087 + + + + + +5367361206712271312 + +simple-shuffle-split + + + +-4202570559885941960->5367361206712271312 + + + + + +-2299645038939435079 + +simple-shuffle-split + + + +-4202570559885941960->-2299645038939435079 + + + + + +-3924030173880096454 + +simple-shuffle-split + + + +-4202570559885941960->-3924030173880096454 + + + + + +-4967372382665515429 + +simple-shuffle-split + + + +-4202570559885941960->-4967372382665515429 + + + + + +-6334242460535207604 + +simple-shuffle-split + + + +-4202570559885941960->-6334242460535207604 + + + + + +7171616034675960405 + +simple-shuffle-split + + + +-4202570559885941960->7171616034675960405 + + + + + +-4313313246320786570 + +simple-shuffle-split + + + +-4202570559885941960->-4313313246320786570 + + + + + +4156444414466270455 + +simple-shuffle-split + + + +-4202570559885941960->4156444414466270455 + + + + + +2615345101919929880 + +simple-shuffle-split + + + +-4202570559885941960->2615345101919929880 + + + + + +-4455614907987618911 + +simple-shuffle-split + + + +-4202570559885941960->-4455614907987618911 + + + + + +1351867867314164930 + +simple-shuffle-split + + + +-4202570559885941960->1351867867314164930 + + + + + +8631376713219557955 + +simple-shuffle-split + + + +-4202570559885941960->8631376713219557955 + + + + + +8206131122471120180 + +simple-shuffle-split + + + +-4202570559885941960->8206131122471120180 + + + + + +-5583222122110866595 + +simple-shuffle-split + + + +-4202570559885941960->-5583222122110866595 + + + + + +-6618863447161603970 + +simple-shuffle-split + + + +-4202570559885941960->-6618863447161603970 + + + + + +-5907855711508110945 + +simple-shuffle-split + + + +-4202570559885941960->-5907855711508110945 + + + + + +-8151482191597418720 + +simple-shuffle-split + + + +-4202570559885941960->-8151482191597418720 + + + + + +1567541030633687305 + +simple-shuffle-split + + + +-4202570559885941960->1567541030633687305 + + + + + +-8279818999500130870 + +simple-shuffle-split + + + +-4202570559885941960->-8279818999500130870 + + + + + +-1358537411616673045 + +simple-shuffle-split + + + +-4202570559885941960->-1358537411616673045 + + + + + +-2486301033570282020 + +simple-shuffle-split + + + +-4202570559885941960->-2486301033570282020 + + + + + +-3346629377195984347 + +simple-shuffle-split + + + +-4202570559885941960->-3346629377195984347 + + + + + +-4390021202761340922 + +simple-shuffle-split + + + +-4202570559885941960->-4390021202761340922 + + + + + +-5464567676767773497 + +simple-shuffle-split + + + +-4202570559885941960->-5464567676767773497 + + + + + +8671026610093107944 + +simple-shuffle-split + + + +-4202570559885941960->8671026610093107944 + + + + + +-136390613181930447 + +simple-shuffle-split + + + +-4202570559885941960->-136390613181930447 + + + + + +-1105374062354319022 + +simple-shuffle-split + + + +-4202570559885941960->-1105374062354319022 + + + + + +-1502474091765884397 + +simple-shuffle-split + + + +-4202570559885941960->-1502474091765884397 + + + + + +6976137968169197828 + +simple-shuffle-split + + + +-4202570559885941960->6976137968169197828 + + + + + +4341882099460429869 + +simple-shuffle-split + + + +-4202570559885941960->4341882099460429869 + + + + + +-7321991368662964722 + +simple-shuffle-split + + + +-4202570559885941960->-7321991368662964722 + + + + + +-8366473760812914897 + +simple-shuffle-split + + + +-4202570559885941960->-8366473760812914897 + + + + + +-859656533121392656 + +simple-shuffle-split + + + +-4202570559885941960->-859656533121392656 + + + + + +7626544253432657369 + +simple-shuffle-split + + + +-4202570559885941960->7626544253432657369 + + + + + +-2772668563804768422 + +simple-shuffle-split + + + +-4202570559885941960->-2772668563804768422 + + + + + +-3817150912723000197 + +simple-shuffle-split + + + +-4202570559885941960->-3817150912723000197 + + + + + +4196112219265108844 + +simple-shuffle-split + + + +-4202570559885941960->4196112219265108844 + + + + + +995700795413278837 + +simple-shuffle-split + + + +-4202570559885941960->995700795413278837 + + + + + +-4202570559880529335 + +44 + + + +3975448567071868207->-4202570559880529335 + + + + + +765402390793559366 + +simple-shuffle-split + + + +-4202570559880529335->765402390793559366 + + + + + +2322314459306295141 + +simple-shuffle-split + + + +-4202570559880529335->2322314459306295141 + + + + + +-1683731778835104984 + +simple-shuffle-split + + + +-4202570559880529335->-1683731778835104984 + + + + + +-1388430963211388409 + +simple-shuffle-split + + + +-4202570559880529335->-1388430963211388409 + + + + + +7546014880032965266 + +simple-shuffle-split + + + +-4202570559880529335->7546014880032965266 + + + + + +7877615050603446641 + +simple-shuffle-split + + + +-4202570559880529335->7877615050603446641 + + + + + +3744679640852546116 + +simple-shuffle-split + + + +-4202570559880529335->3744679640852546116 + + + + + +6472572301351742291 + +simple-shuffle-split + + + +-4202570559880529335->6472572301351742291 + + + + + +-8795344833203558834 + +simple-shuffle-split + + + +-4202570559880529335->-8795344833203558834 + + + + + +-7812920742476832659 + +simple-shuffle-split + + + +-4202570559880529335->-7812920742476832659 + + + + + +4309319220630314032 + +simple-shuffle-split + + + +-4202570559880529335->4309319220630314032 + + + + + +5274012152286026607 + +simple-shuffle-split + + + +-4202570559880529335->5274012152286026607 + + + + + +-4281530416880786534 + +simple-shuffle-split + + + +-4202570559880529335->-4281530416880786534 + + + + + +-4418401631956637159 + +simple-shuffle-split + + + +-4202570559880529335->-4418401631956637159 + + + + + +-7317852986772864084 + +simple-shuffle-split + + + +-4202570559880529335->-7317852986772864084 + + + + + +-5951020374175440709 + +simple-shuffle-split + + + +-4202570559880529335->-5951020374175440709 + + + + + +-4819593501659243050 + +simple-shuffle-split + + + +-4202570559880529335->-4819593501659243050 + + + + + +-2039246511588652875 + +simple-shuffle-split + + + +-4202570559880529335->-2039246511588652875 + + + + + +2183454263025377400 + +simple-shuffle-split + + + +-4202570559880529335->2183454263025377400 + + + + + +2036580565134879575 + +simple-shuffle-split + + + +-4202570559880529335->2036580565134879575 + + + + + +1904738682600066850 + +simple-shuffle-split + + + +-4202570559880529335->1904738682600066850 + + + + + +4710342394373876225 + +simple-shuffle-split + + + +-4202570559880529335->4710342394373876225 + + + + + +-1074558119228731500 + +simple-shuffle-split + + + +-4202570559880529335->-1074558119228731500 + + + + + +-814655595134424925 + +simple-shuffle-split + + + +-4202570559880529335->-814655595134424925 + + + + + +-7676869065391836450 + +simple-shuffle-split + + + +-4202570559880529335->-7676869065391836450 + + + + + +-6713304088032235075 + +simple-shuffle-split + + + +-4202570559880529335->-6713304088032235075 + + + + + +8176237607439769216 + +simple-shuffle-split + + + +-4202570559880529335->8176237607439769216 + + + + + +-8720548214153799425 + +simple-shuffle-split + + + +-4202570559880529335->-8720548214153799425 + + + + + +-9189075339818862550 + +simple-shuffle-split + + + +-4202570559880529335->-9189075339818862550 + + + + + +560601623826610025 + +simple-shuffle-split + + + +-4202570559880529335->560601623826610025 + + + + + +-2260418127744424900 + +simple-shuffle-split + + + +-4202570559880529335->-2260418127744424900 + + + + + +-1353510771596742325 + +simple-shuffle-split + + + +-4202570559880529335->-1353510771596742325 + + + + + +-846771842470991066 + +simple-shuffle-split + + + +-4202570559880529335->-846771842470991066 + + + + + +196624594477661509 + +simple-shuffle-split + + + +-4202570559880529335->196624594477661509 + + + + + +7496510940256654600 + +simple-shuffle-split + + + +-4202570559880529335->7496510940256654600 + + + + + +-1204948791705634841 + +simple-shuffle-split + + + +-4202570559880529335->-1204948791705634841 + + + + + +5385961300589624434 + +simple-shuffle-split + + + +-4202570559880529335->5385961300589624434 + + + + + +-1385564634837282991 + +simple-shuffle-split + + + +-4202570559880529335->-1385564634837282991 + + + + + +-6585158929665508316 + +simple-shuffle-split + + + +-4202570559880529335->-6585158929665508316 + + + + + +4421309312696753459 + +simple-shuffle-split + + + +-4202570559880529335->4421309312696753459 + + + + + +6410687157526530350 + +simple-shuffle-split + + + +-4202570559880529335->6410687157526530350 + + + + + +8033948885261051725 + +simple-shuffle-split + + + +-4202570559880529335->8033948885261051725 + + + + + +4493640861487062800 + +simple-shuffle-split + + + +-4202570559880529335->4493640861487062800 + + + + + +-3406637168494165425 + +simple-shuffle-split + + + +-4202570559880529335->-3406637168494165425 + + + + + +-4026004857989407366 + +simple-shuffle-split + + + +-4202570559880529335->-4026004857989407366 + + + + + +5875617615744044025 + +simple-shuffle-split + + + +-4202570559880529335->5875617615744044025 + + + + + +9078432369254377100 + +simple-shuffle-split + + + +-4202570559880529335->9078432369254377100 + + + + + +520253163814674075 + +simple-shuffle-split + + + +-4202570559880529335->520253163814674075 + + + + + +-4565176265464439882 + +simple-shuffle-split + + + +-4202570559880529335->-4565176265464439882 + + + + + +-4202570559879446810 + +45 + + + +2800444899605902132->-4202570559879446810 + + + + + +2677607782487703655 + +simple-shuffle-split + + + +-4202570559879446810->2677607782487703655 + + + + + +514956353972057480 + +simple-shuffle-split + + + +-4202570559879446810->514956353972057480 + + + + + +4137386294721355205 + +simple-shuffle-split + + + +-4202570559879446810->4137386294721355205 + + + + + +3164926128515845030 + +simple-shuffle-split + + + +-4202570559879446810->3164926128515845030 + + + + + +7152540080132485555 + +simple-shuffle-split + + + +-4202570559879446810->7152540080132485555 + + + + + +5631832359963657380 + +simple-shuffle-split + + + +-4202570559879446810->5631832359963657380 + + + + + +-8604301462073124911 + +simple-shuffle-split + + + +-4202570559879446810->-8604301462073124911 + + + + + +8227087680380607730 + +simple-shuffle-split + + + +-4202570559879446810->8227087680380607730 + + + + + +-7951875832114493745 + +simple-shuffle-split + + + +-4202570559879446810->-7951875832114493745 + + + + + +7334730918875050896 + +simple-shuffle-split + + + +-4202570559879446810->7334730918875050896 + + + + + +-5851682614305490995 + +simple-shuffle-split + + + +-4202570559879446810->-5851682614305490995 + + + + + +-6979468681367090770 + +simple-shuffle-split + + + +-4202570559879446810->-6979468681367090770 + + + + + +4250938072198364955 + +simple-shuffle-split + + + +-4202570559879446810->4250938072198364955 + + + + + +-6637080864966578420 + +simple-shuffle-split + + + +-4202570559879446810->-6637080864966578420 + + + + + +-2682149211510153095 + +simple-shuffle-split + + + +-4202570559879446810->-2682149211510153095 + + + + + +-2468768178378955270 + +simple-shuffle-split + + + +-4202570559879446810->-2468768178378955270 + + + + + +3852760803058918839 + +simple-shuffle-split + + + +-4202570559879446810->3852760803058918839 + + + + + +4070570694097167064 + +simple-shuffle-split + + + +-4202570559879446810->4070570694097167064 + + + + + +9138181466998016789 + +simple-shuffle-split + + + +-4202570559879446810->9138181466998016789 + + + + + +7049866361942057014 + +simple-shuffle-split + + + +-4202570559879446810->7049866361942057014 + + + + + +1075915152220039939 + +simple-shuffle-split + + + +-4202570559879446810->1075915152220039939 + + + + + +-8821277249775408652 + +simple-shuffle-split + + + +-4202570559879446810->-8821277249775408652 + + + + + +-4829218144180173727 + +simple-shuffle-split + + + +-4202570559879446810->-4829218144180173727 + + + + + +-6354393384697048702 + +simple-shuffle-split + + + +-4202570559879446810->-6354393384697048702 + + + + + +-5625004660244326561 + +simple-shuffle-split + + + +-4202570559879446810->-5625004660244326561 + + + + + +-8379935720023823136 + +simple-shuffle-split + + + +-4202570559879446810->-8379935720023823136 + + + + + +-1492443423920592611 + +simple-shuffle-split + + + +-4202570559879446810->-1492443423920592611 + + + + + +-4651457410378817986 + +simple-shuffle-split + + + +-4202570559879446810->-4651457410378817986 + + + + + +533337932371156139 + +simple-shuffle-split + + + +-4202570559879446810->533337932371156139 + + + + + +-518896131897230436 + +simple-shuffle-split + + + +-4202570559879446810->-518896131897230436 + + + + + +1312613996047071689 + +simple-shuffle-split + + + +-4202570559879446810->1312613996047071689 + + + + + +1013687424084856714 + +simple-shuffle-split + + + +-4202570559879446810->1013687424084856714 + + + + + +-5061980660215774585 + +simple-shuffle-split + + + +-4202570559879446810->-5061980660215774585 + + + + + +-6732615328011830360 + +simple-shuffle-split + + + +-4202570559879446810->-6732615328011830360 + + + + + +5283591948668919781 + +simple-shuffle-split + + + +-4202570559879446810->5283591948668919781 + + + + + +4787398001054180806 + +simple-shuffle-split + + + +-4202570559879446810->4787398001054180806 + + + + + +-7803357671313034285 + +simple-shuffle-split + + + +-4202570559879446810->-7803357671313034285 + + + + + +6672190069350831812 + +simple-shuffle-split + + + +-4202570559879446810->6672190069350831812 + + + + + +-6085118702540977679 + +simple-shuffle-split + + + +-4202570559879446810->-6085118702540977679 + + + + + +-8030026473099779310 + +simple-shuffle-split + + + +-4202570559879446810->-8030026473099779310 + + + + + +961134422214632431 + +simple-shuffle-split + + + +-4202570559879446810->961134422214632431 + + + + + +8370274173611549872 + +simple-shuffle-split + + + +-4202570559879446810->8370274173611549872 + + + + + +-4777252700452064019 + +simple-shuffle-split + + + +-4202570559879446810->-4777252700452064019 + + + + + +3687924390134515406 + +simple-shuffle-split + + + +-4202570559879446810->3687924390134515406 + + + + + +7631888851942998331 + +simple-shuffle-split + + + +-4202570559879446810->7631888851942998331 + + + + + +6667236794196954156 + +simple-shuffle-split + + + +-4202570559879446810->6667236794196954156 + + + + + +703267963432022681 + +simple-shuffle-split + + + +-4202570559879446810->703267963432022681 + + + + + +-279138258149790694 + +simple-shuffle-split + + + +-4202570559879446810->-279138258149790694 + + + + + +-2396436594116466729 + +simple-shuffle-split + + + +-4202570559879446810->-2396436594116466729 + + + + + +-4202570559882694385 + +46 + + + +-3574868481938336859->-4202570559882694385 + + + + + +-539930752842309400 + +simple-shuffle-split + + + +-4202570559882694385->-539930752842309400 + + + + + +1693997291722226375 + +simple-shuffle-split + + + +-4202570559882694385->1693997291722226375 + + + + + +975238977993362950 + +simple-shuffle-split + + + +-4202570559882694385->975238977993362950 + + + + + +2018630839030898725 + +simple-shuffle-split + + + +-4202570559882694385->2018630839030898725 + + + + + +5199941565409328900 + +simple-shuffle-split + + + +-4202570559882694385->5199941565409328900 + + + + + +6243283774177427475 + +simple-shuffle-split + + + +-4202570559882694385->6243283774177427475 + + + + + +8946100326454656850 + +simple-shuffle-split + + + +-4202570559882694385->8946100326454656850 + + + + + +9099433518524248625 + +simple-shuffle-split + + + +-4202570559882694385->9099433518524248625 + + + + + +6276728663850808816 + +simple-shuffle-split + + + +-4202570559882694385->6276728663850808816 + + + + + +-620716996724169425 + +simple-shuffle-split + + + +-4202570559882694385->-620716996724169425 + + + + + +-9098224101585368050 + +simple-shuffle-split + + + +-4202570559882694385->-9098224101585368050 + + + + + +-8739601758022238675 + +simple-shuffle-split + + + +-4202570559882694385->-8739601758022238675 + + + + + +-7695086518668990100 + +simple-shuffle-split + + + +-4202570559882694385->-7695086518668990100 + + + + + +-3287667131647784325 + +simple-shuffle-split + + + +-4202570559882694385->-3287667131647784325 + + + + + +-2314119811946422950 + +simple-shuffle-split + + + +-4202570559882694385->-2314119811946422950 + + + + + +-2456565772629031975 + +simple-shuffle-split + + + +-4202570559882694385->-2456565772629031975 + + + + + +2886189140323747384 + +simple-shuffle-split + + + +-4202570559882694385->2886189140323747384 + + + + + +4003953683735367959 + +simple-shuffle-split + + + +-4202570559882694385->4003953683735367959 + + + + + +7110873859703306134 + +simple-shuffle-split + + + +-4202570559882694385->7110873859703306134 + + + + + +8154265755035233909 + +simple-shuffle-split + + + +-4202570559882694385->8154265755035233909 + + + + + +7434671350715084884 + +simple-shuffle-split + + + +-4202570559882694385->7434671350715084884 + + + + + +8479153742847714659 + +simple-shuffle-split + + + +-4202570559882694385->8479153742847714659 + + + + + +-6866207458507005982 + +simple-shuffle-split + + + +-4202570559882694385->-6866207458507005982 + + + + + +-5738475575268415807 + +simple-shuffle-split + + + +-4202570559882694385->-5738475575268415807 + + + + + +-8302828196307841216 + +simple-shuffle-split + + + +-4202570559882694385->-8302828196307841216 + + + + + +-6679587751915480641 + +simple-shuffle-split + + + +-4202570559882694385->-6679587751915480641 + + + + + +-5635105395237709666 + +simple-shuffle-split + + + +-4202570559882694385->-5635105395237709666 + + + + + +-3682133972012738691 + +simple-shuffle-split + + + +-4202570559882694385->-3682133972012738691 + + + + + +-2637651623059866116 + +simple-shuffle-split + + + +-4202570559882694385->-2637651623059866116 + + + + + +-2279046006932842741 + +simple-shuffle-split + + + +-4202570559882694385->-2279046006932842741 + + + + + +-44314832047890966 + +simple-shuffle-split + + + +-4202570559882694385->-44314832047890966 + + + + + +3076277573217268009 + +simple-shuffle-split + + + +-4202570559882694385->3076277573217268009 + + + + + +-3028074845661880632 + +simple-shuffle-split + + + +-4202570559882694385->-3028074845661880632 + + + + + +-8835516757013226073 + +simple-shuffle-split + + + +-4202570559882694385->-8835516757013226073 + + + + + +-9101144971356438298 + +simple-shuffle-split + + + +-4202570559882694385->-9101144971356438298 + + + + + +868108768142418693 + +simple-shuffle-split + + + +-4202570559882694385->868108768142418693 + + + + + +-4876437806920849948 + +simple-shuffle-split + + + +-4202570559882694385->-4876437806920849948 + + + + + +-3831958854481096973 + +simple-shuffle-split + + + +-4202570559882694385->-3831958854481096973 + + + + + +-4538233602977531598 + +simple-shuffle-split + + + +-4202570559882694385->-4538233602977531598 + + + + + +5422144383104101393 + +simple-shuffle-split + + + +-4202570559882694385->5422144383104101393 + + + + + +5786986866200133584 + +simple-shuffle-split + + + +-4202570559882694385->5786986866200133584 + + + + + +-2812401674582819057 + +simple-shuffle-split + + + +-4202570559882694385->-2812401674582819057 + + + + + +-578777687284708882 + +simple-shuffle-split + + + +-4202570559882694385->-578777687284708882 + + + + + +-219051527100267507 + +simple-shuffle-split + + + +-4202570559882694385->-219051527100267507 + + + + + +-8697663585944164532 + +simple-shuffle-split + + + +-4202570559882694385->-8697663585944164532 + + + + + +2778388888606999643 + +simple-shuffle-split + + + +-4202570559882694385->2778388888606999643 + + + + + +3826192958802057018 + +simple-shuffle-split + + + +-4202570559882694385->3826192958802057018 + + + + + +4251433999186967993 + +simple-shuffle-split + + + +-4202570559882694385->4251433999186967993 + + + + + +805308981296170008 + +simple-shuffle-split + + + +-4202570559882694385->805308981296170008 + + + + + +-4202570559881611860 + +47 + + + +-4601145275586085734->-4202570559881611860 + + + + + +-7142186444478118343 + +simple-shuffle-split + + + +-4202570559881611860->-7142186444478118343 + + + + + +-7477108251005134918 + +simple-shuffle-split + + + +-4202570559881611860->-7477108251005134918 + + + + + +-8521590608791411493 + +simple-shuffle-split + + + +-4202570559881611860->-8521590608791411493 + + + + + +-1756452960081826868 + +simple-shuffle-split + + + +-4202570559881611860->-1756452960081826868 + + + + + +5753685901314578573 + +simple-shuffle-split + + + +-4202570559881611860->5753685901314578573 + + + + + +4188986975589836398 + +simple-shuffle-split + + + +-4202570559881611860->4188986975589836398 + + + + + +-6899526248249210993 + +simple-shuffle-split + + + +-4202570559881611860->-6899526248249210993 + + + + + +1739889128364167248 + +simple-shuffle-split + + + +-4202570559881611860->1739889128364167248 + + + + + +-4289661086082239343 + +simple-shuffle-split + + + +-4202570559881611860->-4289661086082239343 + + + + + +-4466249485222064718 + +simple-shuffle-split + + + +-4202570559881611860->-4466249485222064718 + + + + + +-6156300142636951693 + +simple-shuffle-split + + + +-4202570559881611860->-6156300142636951693 + + + + + +-7128702755145107868 + +simple-shuffle-split + + + +-4202570559881611860->-7128702755145107868 + + + + + +-3289950943374392443 + +simple-shuffle-split + + + +-4202570559881611860->-3289950943374392443 + + + + + +-4262407703142193818 + +simple-shuffle-split + + + +-4202570559881611860->-4262407703142193818 + + + + + +-5298031160156523993 + +simple-shuffle-split + + + +-4202570559881611860->-5298031160156523993 + + + + + +-5542142251584758968 + +simple-shuffle-split + + + +-4202570559881611860->-5542142251584758968 + + + + + +8133963301538507657 + +simple-shuffle-split + + + +-4202570559881611860->8133963301538507657 + + + + + +5406070676528811082 + +simple-shuffle-split + + + +-4202570559881611860->5406070676528811082 + + + + + +3988098037103008107 + +simple-shuffle-split + + + +-4202570559881611860->3988098037103008107 + + + + + +2422349512960957532 + +simple-shuffle-split + + + +-4202570559881611860->2422349512960957532 + + + + + +1375515459401134557 + +simple-shuffle-split + + + +-4202570559881611860->1375515459401134557 + + + + + +-26802553559900418 + +simple-shuffle-split + + + +-4202570559881611860->-26802553559900418 + + + + + +-2757046884319693793 + +simple-shuffle-split + + + +-4202570559881611860->-2757046884319693793 + + + + + +-2620134805293404768 + +simple-shuffle-split + + + +-4202570559881611860->-2620134805293404768 + + + + + +308684571650206241 + +simple-shuffle-split + + + +-4202570559881611860->308684571650206241 + + + + + +-599331097234822334 + +simple-shuffle-split + + + +-4202570559881611860->-599331097234822334 + + + + + +-4981143725604578109 + +simple-shuffle-split + + + +-4202570559881611860->-4981143725604578109 + + + + + +-5954691009816439884 + +simple-shuffle-split + + + +-4202570559881611860->-5954691009816439884 + + + + + +-6989210575974223659 + +simple-shuffle-split + + + +-4202570559881611860->-6989210575974223659 + + + + + +-8604685429799873034 + +simple-shuffle-split + + + +-4202570559881611860->-8604685429799873034 + + + + + +-8907769735030224009 + +simple-shuffle-split + + + +-4202570559881611860->-8907769735030224009 + + + + + +8017120767339515032 + +simple-shuffle-split + + + +-4202570559881611860->8017120767339515032 + + + + + +2315628964148566105 + +simple-shuffle-split + + + +-4202570559881611860->2315628964148566105 + + + + + +-7665921284939227686 + +simple-shuffle-split + + + +-4202570559881611860->-7665921284939227686 + + + + + +-7519046458590029061 + +simple-shuffle-split + + + +-4202570559881611860->-7519046458590029061 + + + + + +8279630784040153580 + +simple-shuffle-split + + + +-4202570559881611860->8279630784040153580 + + + + + +-2726837458304787795 + +simple-shuffle-split + + + +-4202570559881611860->-2726837458304787795 + + + + + +-2025851282881958770 + +simple-shuffle-split + + + +-4202570559881611860->-2025851282881958770 + + + + + +6452765326342785455 + +simple-shuffle-split + + + +-4202570559881611860->6452765326342785455 + + + + + +3694840401406023280 + +simple-shuffle-split + + + +-4202570559881611860->3694840401406023280 + + + + + +5086662037790259889 + +simple-shuffle-split + + + +-4202570559881611860->5086662037790259889 + + + + + +-4733951119037603886 + +simple-shuffle-split + + + +-4202570559881611860->-4733951119037603886 + + + + + +-6340586559619754861 + +simple-shuffle-split + + + +-4202570559881611860->-6340586559619754861 + + + + + +2138030048513804164 + +simple-shuffle-split + + + +-4202570559881611860->2138030048513804164 + + + + + +607322113550106789 + +simple-shuffle-split + + + +-4202570559881611860->607322113550106789 + + + + + +8047611868749636230 + +simple-shuffle-split + + + +-4202570559881611860->8047611868749636230 + + + + + +7670152749488931655 + +simple-shuffle-split + + + +-4202570559881611860->7670152749488931655 + + + + + +-2227311892354026136 + +simple-shuffle-split + + + +-4202570559881611860->-2227311892354026136 + + + + + +-2236560016615984727 + +simple-shuffle-split + + + +-4202570559881611860->-2236560016615984727 + + + + + +-4202570559858878835 + +48 + + + +4802138345093437923->-4202570559858878835 + + + + + +3549443367749063850 + +simple-shuffle-split + + + +-4202570559858878835->3549443367749063850 + + + + + +-3366530374204935191 + +simple-shuffle-split + + + +-4202570559858878835->-3366530374204935191 + + + + + +2788302523013543100 + +simple-shuffle-split + + + +-4202570559858878835->2788302523013543100 + + + + + +2506069369240048075 + +simple-shuffle-split + + + +-4202570559858878835->2506069369240048075 + + + + + +-371220232816908450 + +simple-shuffle-split + + + +-4202570559858878835->-371220232816908450 + + + + + +-7881337794243569091 + +simple-shuffle-split + + + +-4202570559858878835->-7881337794243569091 + + + + + +-1682464094607699200 + +simple-shuffle-split + + + +-4202570559858878835->-1682464094607699200 + + + + + +-708916802584337025 + +simple-shuffle-split + + + +-4202570559858878835->-708916802584337025 + + + + + +6254810874541667234 + +simple-shuffle-split + + + +-4202570559858878835->6254810874541667234 + + + + + +-1032448542788062591 + +simple-shuffle-split + + + +-4202570559858878835->-1032448542788062591 + + + + + +-3901867643125953516 + +simple-shuffle-split + + + +-4202570559858878835->-3901867643125953516 + + + + + +-3499332406775237341 + +simple-shuffle-split + + + +-4202570559858878835->-3499332406775237341 + + + + + +1775453080013454934 + +simple-shuffle-split + + + +-4202570559858878835->1775453080013454934 + + + + + +-4878700316252415691 + +simple-shuffle-split + + + +-4202570559858878835->-4878700316252415691 + + + + + +7720910888120907000 + +simple-shuffle-split + + + +-4202570559858878835->7720910888120907000 + + + + + +-9160047205025049641 + +simple-shuffle-split + + + +-4202570559858878835->-9160047205025049641 + + + + + +-7664362077485643750 + +simple-shuffle-split + + + +-4202570559858878835->-7664362077485643750 + + + + + +2826141082157567641 + +simple-shuffle-split + + + +-4202570559858878835->2826141082157567641 + + + + + +-2911076095335651284 + +simple-shuffle-split + + + +-4202570559858878835->-2911076095335651284 + + + + + +8619695379306495291 + +simple-shuffle-split + + + +-4202570559858878835->8619695379306495291 + + + + + +5779795272075522766 + +simple-shuffle-split + + + +-4202570559858878835->5779795272075522766 + + + + + +-2768630198877085459 + +simple-shuffle-split + + + +-4202570559858878835->-2768630198877085459 + + + + + +-6757349109966991184 + +simple-shuffle-split + + + +-4202570559858878835->-6757349109966991184 + + + + + +4179256065765470191 + +simple-shuffle-split + + + +-4202570559858878835->4179256065765470191 + + + + + +-7042257413225562350 + +simple-shuffle-split + + + +-4202570559858878835->-7042257413225562350 + + + + + +-6068710119019829775 + +simple-shuffle-split + + + +-4202570559858878835->-6068710119019829775 + + + + + +-131318039171779900 + +simple-shuffle-split + + + +-4202570559858878835->-131318039171779900 + + + + + +913165454815416275 + +simple-shuffle-split + + + +-4202570559858878835->913165454815416275 + + + + + +8079905664587764166 + +simple-shuffle-split + + + +-4202570559858878835->8079905664587764166 + + + + + +7375463930247482341 + +simple-shuffle-split + + + +-4202570559858878835->7375463930247482341 + + + + + +-4685065275656845400 + +simple-shuffle-split + + + +-4202570559858878835->-4685065275656845400 + + + + + +7107466729693933191 + +simple-shuffle-split + + + +-4202570559858878835->7107466729693933191 + + + + + +4231150082615438218 + +simple-shuffle-split + + + +-4202570559858878835->4231150082615438218 + + + + + +-4478932641559648823 + +simple-shuffle-split + + + +-4202570559858878835->-4478932641559648823 + + + + + +-7286971967421442148 + +simple-shuffle-split + + + +-4202570559858878835->-7286971967421442148 + + + + + +-6305675238061573973 + +simple-shuffle-split + + + +-4202570559858878835->-6305675238061573973 + + + + + +5804426144232147518 + +simple-shuffle-split + + + +-4202570559858878835->5804426144232147518 + + + + + +6781316382054417693 + +simple-shuffle-split + + + +-4202570559858878835->6781316382054417693 + + + + + +3193339738691385568 + +simple-shuffle-split + + + +-4202570559858878835->3193339738691385568 + + + + + +5426967094772654943 + +simple-shuffle-split + + + +-4202570559858878835->5426967094772654943 + + + + + +5253338724633865602 + +simple-shuffle-split + + + +-4202570559858878835->5253338724633865602 + + + + + +8054518117865697377 + +simple-shuffle-split + + + +-4202570559858878835->8054518117865697377 + + + + + +2274079379209291252 + +simple-shuffle-split + + + +-4202570559858878835->2274079379209291252 + + + + + +-5692125415090002173 + +simple-shuffle-split + + + +-4202570559858878835->-5692125415090002173 + + + + + +-1901834945792586698 + +simple-shuffle-split + + + +-4202570559858878835->-1901834945792586698 + + + + + +1199532914036262677 + +simple-shuffle-split + + + +-4202570559858878835->1199532914036262677 + + + + + +-4002002313216356648 + +simple-shuffle-split + + + +-4202570559858878835->-4002002313216356648 + + + + + +-2946313899392806473 + +simple-shuffle-split + + + +-4202570559858878835->-2946313899392806473 + + + + + +-9197922129803787782 + +simple-shuffle-split + + + +-4202570559858878835->-9197922129803787782 + + + + + +-6847291331795655860 + +(0, 0) + + + +1484894084834626618->-6847291331795655860 + + + + + +1491115096712983436 + +simple-shuffle-combine + + + +-6847291331795655860->1491115096712983436 + + + + + +-6847292592992654683 + +(1, 0) + + + +-6326953721752775749->-6847292592992654683 + + + + + +-6847292592992654683->1491115096712983436 + + + + + +-6847293854189653506 + +(2, 0) + + + +-7153607758011772404->-6847293854189653506 + + + + + +-6847293854189653506->1491115096712983436 + + + + + +-6847295115386652329 + +(3, 0) + + + +-4972632081319006931->-6847295115386652329 + + + + + +-6847295115386652329->1491115096712983436 + + + + + +-6847296376583651152 + +(4, 0) + + + +-5766172080440701586->-6847296376583651152 + + + + + +-6847296376583651152->1491115096712983436 + + + + + +-6847297637780649975 + +(5, 0) + + + +4692831239999388911->-6847297637780649975 + + + + + +-6847297637780649975->1491115096712983436 + + + + + +-6847298898977648798 + +(6, 0) + + + +-7227592232223727344->-6847298898977648798 + + + + + +-6847298898977648798->1491115096712983436 + + + + + +-6847300160174647621 + +(7, 0) + + + +-5754122772232681887->-6847300160174647621 + + + + + +-6847300160174647621->1491115096712983436 + + + + + +-6847281242219665276 + +(8, 0) + + + +4857179192858259762->-6847281242219665276 + + + + + +-6847281242219665276->1491115096712983436 + + + + + +-6847282503416664099 + +(9, 0) + + + +-1919952402746357421->-6847282503416664099 + + + + + +-6847282503416664099->1491115096712983436 + + + + + +-6847283764613662922 + +(10, 0) + + + +-9081145470470620508->-6847283764613662922 + + + + + +-6847283764613662922->1491115096712983436 + + + + + +-6847285025810661745 + +(11, 0) + + + +-6890795871647074779->-6847285025810661745 + + + + + +-6847285025810661745->1491115096712983436 + + + + + +-6847286287007660568 + +(12, 0) + + + +-8195566856495047834->-6847286287007660568 + + + + + +-6847286287007660568->1491115096712983436 + + + + + +-6847287548204659391 + +(13, 0) + + + +-6292182916893524345->-6847287548204659391 + + + + + +-6847287548204659391->1491115096712983436 + + + + + +-6847288809401658214 + +(14, 0) + + + +8927323017630980616->-6847288809401658214 + + + + + +-6847288809401658214->1491115096712983436 + + + + + +-6847290070598657037 + +(15, 0) + + + +3343389683881411673->-6847290070598657037 + + + + + +-6847290070598657037->1491115096712983436 + + + + + +-6847271152643674692 + +(16, 0) + + + +4059132380626355914->-6847271152643674692 + + + + + +-6847271152643674692->1491115096712983436 + + + + + +-6847272413840673515 + +(17, 0) + + + +-1393577554088563125->-6847272413840673515 + + + + + +-6847272413840673515->1491115096712983436 + + + + + +-6847273675037672338 + +(18, 0) + + + +-4999527107595264868->-6847273675037672338 + + + + + +-6847273675037672338->1491115096712983436 + + + + + +-6847274936234671161 + +(19, 0) + + + +-4361290628027064195->-6847274936234671161 + + + + + +-6847274936234671161->1491115096712983436 + + + + + +-6847276197431669984 + +(20, 0) + + + +-5059713591180130306->-6847276197431669984 + + + + + +-6847276197431669984->1491115096712983436 + + + + + +-6847277458628668807 + +(21, 0) + + + +-2896326221037548417->-6847277458628668807 + + + + + +-6847277458628668807->1491115096712983436 + + + + + +-6847278719825667630 + +(22, 0) + + + +-4923351608660794272->-6847278719825667630 + + + + + +-6847278719825667630->1491115096712983436 + + + + + +-6847279981022666453 + +(23, 0) + + + +-3066390384665253455->-6847279981022666453 + + + + + +-6847279981022666453->1491115096712983436 + + + + + +-6847261063067684108 + +(24, 0) + + + +5995923826237597890->-6847261063067684108 + + + + + +-6847261063067684108->1491115096712983436 + + + + + +-6847262324264682931 + +(25, 0) + + + +-1741569752342307677->-6847262324264682931 + + + + + +-6847262324264682931->1491115096712983436 + + + + + +-6847263585461681754 + +(26, 0) + + + +-5038466543536177676->-6847263585461681754 + + + + + +-6847263585461681754->1491115096712983436 + + + + + +-6847264846658680577 + +(27, 0) + + + +8604617348867777141->-6847264846658680577 + + + + + +-6847264846658680577->1491115096712983436 + + + + + +-6847266107855679400 + +(28, 0) + + + +5925664213485840886->-6847266107855679400 + + + + + +-6847266107855679400->1491115096712983436 + + + + + +-6847267369052678223 + +(29, 0) + + + +-272741049218666025->-6847267369052678223 + + + + + +-6847267369052678223->1491115096712983436 + + + + + +-6847268630249677046 + +(30, 0) + + + +5096749064732241240->-6847268630249677046 + + + + + +-6847268630249677046->1491115096712983436 + + + + + +-6847269891446675869 + +(31, 0) + + + +-3501878531947119703->-6847269891446675869 + + + + + +-6847269891446675869->1491115096712983436 + + + + + +-6847250973491693524 + +(32, 0) + + + +482769098229217818->-6847250973491693524 + + + + + +-6847250973491693524->1491115096712983436 + + + + + +-6847252234688692347 + +(33, 0) + + + +2630658632484823451->-6847252234688692347 + + + + + +-6847252234688692347->1491115096712983436 + + + + + +-6847253495885691170 + +(34, 0) + + + +6683614170775187436->-6847253495885691170 + + + + + +-6847253495885691170->1491115096712983436 + + + + + +-6847254757082689993 + +(35, 0) + + + +-8374219548216852723->-6847254757082689993 + + + + + +-6847254757082689993->1491115096712983436 + + + + + +-6847256018279688816 + +(36, 0) + + + +-3758663352665206962->-6847256018279688816 + + + + + +-6847256018279688816->1491115096712983436 + + + + + +-6847257279476687639 + +(37, 0) + + + +-1446940585988295729->-6847257279476687639 + + + + + +-6847257279476687639->1491115096712983436 + + + + + +-6847258540673686462 + +(38, 0) + + + +-2275926841802611984->-6847258540673686462 + + + + + +-6847258540673686462->1491115096712983436 + + + + + +-6847259801870685285 + +(39, 0) + + + +-7272794668948052927->-6847259801870685285 + + + + + +-6847259801870685285->1491115096712983436 + + + + + +-6847240883915702940 + +(40, 0) + + + +-354591326279656430->-6847240883915702940 + + + + + +-6847240883915702940->1491115096712983436 + + + + + +-6847242145112701763 + +(41, 0) + + + +1770010154810922803->-6847242145112701763 + + + + + +-6847242145112701763->1491115096712983436 + + + + + +-6847243406309700586 + +(42, 0) + + + +8386346349976722564->-6847243406309700586 + + + + + +-6847243406309700586->1491115096712983436 + + + + + +-6847244667506699409 + +(43, 0) + + + +2170820924475612421->-6847244667506699409 + + + + + +-6847244667506699409->1491115096712983436 + + + + + +-6847245928703698232 + +(44, 0) + + + +765402390793559366->-6847245928703698232 + + + + + +-6847245928703698232->1491115096712983436 + + + + + +-6847247189900697055 + +(45, 0) + + + +2677607782487703655->-6847247189900697055 + + + + + +-6847247189900697055->1491115096712983436 + + + + + +-6847248451097695878 + +(46, 0) + + + +-539930752842309400->-6847248451097695878 + + + + + +-6847248451097695878->1491115096712983436 + + + + + +-6847249712294694701 + +(47, 0) + + + +-7142186444478118343->-6847249712294694701 + + + + + +-6847249712294694701->1491115096712983436 + + + + + +-6847230794339712356 + +(48, 0) + + + +3549443367749063850->-6847230794339712356 + + + + + +-6847230794339712356->1491115096712983436 + + + + + +-6847291331796820909 + +(0, 1) + + + +-7550883156245396807->-6847291331796820909 + + + + + +1237898002951376167 + +simple-shuffle-combine + + + +-6847291331796820909->1237898002951376167 + + + + + +-6847292592991489634 + +(1, 1) + + + +-8991661886735607124->-6847292592991489634 + + + + + +-6847292592991489634->1237898002951376167 + + + + + +-6847293854190818555 + +(2, 1) + + + +2671739413575161371->-6847293854190818555 + + + + + +-6847293854190818555->1237898002951376167 + + + + + +-6847295115385487280 + +(3, 1) + + + +-4782560445697445106->-6847295115385487280 + + + + + +-6847295115385487280->1237898002951376167 + + + + + +-6847296376584816201 + +(4, 1) + + + +-6523831002617041011->-6847296376584816201 + + + + + +-6847296376584816201->1237898002951376167 + + + + + +-6847297637779484926 + +(5, 1) + + + +-6866373309042461264->-6847297637779484926 + + + + + +-6847297637779484926->1237898002951376167 + + + + + +-6847298898978813847 + +(6, 1) + + + +3712333548520420431->-6847298898978813847 + + + + + +-6847298898978813847->1237898002951376167 + + + + + +-6847300160173482572 + +(7, 1) + + + +-6797501304529601662->-6847300160173482572 + + + + + +-6847300160173482572->1237898002951376167 + + + + + +-6847281242220830325 + +(8, 1) + + + +-820768228302314479->-6847281242220830325 + + + + + +-6847281242220830325->1237898002951376167 + + + + + +-6847282503415499050 + +(9, 1) + + + +-2973331147586494396->-6847282503415499050 + + + + + +-6847282503415499050->1237898002951376167 + + + + + +-6847283764614827971 + +(10, 1) + + + +-8104277598983161933->-6847283764614827971 + + + + + +-6847283764614827971->1237898002951376167 + + + + + +-6847285025809496696 + +(11, 1) + + + +-7140679721968040954->-6847285025809496696 + + + + + +-6847285025809496696->1237898002951376167 + + + + + +-6847286287008825617 + +(12, 1) + + + +-5127127744061390459->-6847286287008825617 + + + + + +-6847286287008825617->1237898002951376167 + + + + + +-6847287548203494342 + +(13, 1) + + + +-8977249768265132120->-6847287548203494342 + + + + + +-6847287548203494342->1237898002951376167 + + + + + +-6847288809402823263 + +(14, 1) + + + +-8391689175056992025->-6847288809402823263 + + + + + +-6847288809402823263->1237898002951376167 + + + + + +-6847290070597491988 + +(15, 1) + + + +2378696759915956698->-6847290070597491988 + + + + + +-6847290070597491988->1237898002951376167 + + + + + +-6847271152644839741 + +(16, 1) + + + +4405423563685434889->-6847271152644839741 + + + + + +-6847271152644839741->1237898002951376167 + + + + + +-6847272413839508466 + +(17, 1) + + + +6116561309507533116->-6847272413839508466 + + + + + +-6847272413839508466->1237898002951376167 + + + + + +-6847273675038837387 + +(18, 1) + + + +4969709886731396523->-6847273675038837387 + + + + + +-6847273675038837387->1237898002951376167 + + + + + +-6847274936233506112 + +(19, 1) + + + +-5480181371005071970->-6847274936233506112 + + + + + +-6847274936233506112->1237898002951376167 + + + + + +-6847276197432835033 + +(20, 1) + + + +-3538987973062564131->-6847276197432835033 + + + + + +-6847276197432835033->1237898002951376167 + + + + + +-6847277458627503758 + +(21, 1) + + + +-3939704761128648192->-6847277458627503758 + + + + + +-6847277458627503758->1237898002951376167 + + + + + +-6847278719826832679 + +(22, 1) + + + +-3950912682565726497->-6847278719826832679 + + + + + +-6847278719826832679->1237898002951376167 + + + + + +-6847279981021501404 + +(23, 1) + + + +5411117864173724370->-6847279981021501404 + + + + + +-6847279981021501404->1237898002951376167 + + + + + +-6847261063068849157 + +(24, 1) + + + +8736130909828188065->-6847261063068849157 + + + + + +-6847261063068849157->1237898002951376167 + + + + + +-6847262324263517882 + +(25, 1) + + + +6707014549213152148->-6847262324263517882 + + + + + +-6847262324263517882->1237898002951376167 + + + + + +-6847263585462846803 + +(26, 1) + + + +-3965064828788072701->-6847263585462846803 + + + + + +-6847263585462846803->1237898002951376167 + + + + + +-6847264846657515528 + +(27, 1) + + + +7060175082728348566->-6847264846657515528 + + + + + +-6847264846657515528->1237898002951376167 + + + + + +-6847266107856844449 + +(28, 1) + + + +8681683999765275861->-6847266107856844449 + + + + + +-6847266107856844449->1237898002951376167 + + + + + +-6847267369051513174 + +(29, 1) + + + +-497106873314702600->-6847267369051513174 + + + + + +-6847267369051513174->1237898002951376167 + + + + + +-6847268630250842095 + +(30, 1) + + + +5395371549533020215->-6847268630250842095 + + + + + +-6847268630250842095->1237898002951376167 + + + + + +-6847269891445510820 + +(31, 1) + + + +-2642126062111939478->-6847269891445510820 + + + + + +-6847269891445510820->1237898002951376167 + + + + + +-6847250973492858573 + +(32, 1) + + + +2169194037029437593->-6847250973492858573 + + + + + +-6847250973492858573->1237898002951376167 + + + + + +-6847252234687527298 + +(33, 1) + + + +-7220129031148890740->-6847252234687527298 + + + + + +-6847252234687527298->1237898002951376167 + + + + + +-6847253495886856219 + +(34, 1) + + + +7758124251451586811->-6847253495886856219 + + + + + +-6847253495886856219->1237898002951376167 + + + + + +-6847254757081524944 + +(35, 1) + + + +-2700621368800260882->-6847254757081524944 + + + + + +-6847254757081524944->1237898002951376167 + + + + + +-6847256018280853865 + +(36, 1) + + + +7882837374970216813->-6847256018280853865 + + + + + +-6847256018280853865->1237898002951376167 + + + + + +-6847257279475522590 + +(37, 1) + + + +-1291207986217345904->-6847257279475522590 + + + + + +-6847257279475522590->1237898002951376167 + + + + + +-6847258540674851511 + +(38, 1) + + + +-2501523671008416209->-6847258540674851511 + + + + + +-6847258540674851511->1237898002951376167 + + + + + +-6847259801869520236 + +(39, 1) + + + +-1545146796184432286->-6847259801869520236 + + + + + +-6847259801869520236->1237898002951376167 + + + + + +-6847240883916867989 + +(40, 1) + + + +1185367853893410545->-6847240883916867989 + + + + + +-6847240883916867989->1237898002951376167 + + + + + +-6847242145111536714 + +(41, 1) + + + +-9165486688265732572->-6847242145111536714 + + + + + +-6847242145111536714->1237898002951376167 + + + + + +-6847243406310865635 + +(42, 1) + + + +-8574851450908231277->-6847243406310865635 + + + + + +-6847243406310865635->1237898002951376167 + + + + + +-6847244667505534360 + +(43, 1) + + + +1745594897655907046->-6847244667505534360 + + + + + +-6847244667505534360->1237898002951376167 + + + + + +-6847245928704863281 + +(44, 1) + + + +2322314459306295141->-6847245928704863281 + + + + + +-6847245928704863281->1237898002951376167 + + + + + +-6847247189899532006 + +(45, 1) + + + +514956353972057480->-6847247189899532006 + + + + + +-6847247189899532006->1237898002951376167 + + + + + +-6847248451098860927 + +(46, 1) + + + +1693997291722226375->-6847248451098860927 + + + + + +-6847248451098860927->1237898002951376167 + + + + + +-6847249712293529652 + +(47, 1) + + + +-7477108251005134918->-6847249712293529652 + + + + + +-6847249712293529652->1237898002951376167 + + + + + +-6847230794340877405 + +(48, 1) + + + +-3366530374204935191->-6847230794340877405 + + + + + +-6847230794340877405->1237898002951376167 + + + + + +-6847291331793325762 + +(0, 2) + + + +7394493774267332684->-6847291331793325762 + + + + + +2268567409594649842 + +simple-shuffle-combine + + + +-6847291331793325762->2268567409594649842 + + + + + +-6847292592994984781 + +(1, 2) + + + +-3365412286419040999->-6847292592994984781 + + + + + +-6847292592994984781->2268567409594649842 + + + + + +-6847293854191983604 + +(2, 2) + + + +-5310255680497480454->-6847293854191983604 + + + + + +-6847293854191983604->2268567409594649842 + + + + + +-6847295115384322231 + +(3, 2) + + + +-9093401613779145681->-6847295115384322231 + + + + + +-6847295115384322231->2268567409594649842 + + + + + +-6847296376581321054 + +(4, 2) + + + +8534707863943084880->-6847296376581321054 + + + + + +-6847296376581321054->2268567409594649842 + + + + + +-6847297637782980073 + +(5, 2) + + + +-2289027599731397139->-6847297637782980073 + + + + + +-6847297637782980073->2268567409594649842 + + + + + +-6847298898979978896 + +(6, 2) + + + +4756817040307925806->-6847298898979978896 + + + + + +-6847298898979978896->2268567409594649842 + + + + + +-6847300160172317523 + +(7, 2) + + + +-8329741654296963837->-6847300160172317523 + + + + + +-6847300160172317523->2268567409594649842 + + + + + +-6847281242217335178 + +(8, 2) + + + +-7800744669336511004->-6847281242217335178 + + + + + +-6847281242217335178->2268567409594649842 + + + + + +-6847282503418994197 + +(9, 2) + + + +1239552412960182129->-6847282503418994197 + + + + + +-6847282503418994197->2268567409594649842 + + + + + +-6847283764615993020 + +(10, 2) + + + +3631655476075875058->-6847283764615993020 + + + + + +-6847283764615993020->2268567409594649842 + + + + + +-6847285025808331647 + +(11, 2) + + + +8996942618686527687->-6847285025808331647 + + + + + +-6847285025808331647->2268567409594649842 + + + + + +-6847286287005330470 + +(12, 2) + + + +7657840427328354632->-6847286287005330470 + + + + + +-6847286287005330470->2268567409594649842 + + + + + +-6847287548206989489 + +(13, 2) + + + +-3240011315206611995->-6847287548206989489 + + + + + +-6847287548206989489->2268567409594649842 + + + + + +-6847288809403988312 + +(14, 2) + + + +-6355486169794495450->-6847288809403988312 + + + + + +-6847288809403988312->2268567409594649842 + + + + + +-6847290070596326939 + +(15, 2) + + + +894255442130601723->-6847290070596326939 + + + + + +-6847290070596326939->2268567409594649842 + + + + + +-6847271152641344594 + +(16, 2) + + + +895202032546886364->-6847271152641344594 + + + + + +-6847271152641344594->2268567409594649842 + + + + + +-6847272413843003613 + +(17, 2) + + + +-8377379541055252375->-6847272413843003613 + + + + + +-6847272413843003613->2268567409594649842 + + + + + +-6847273675040002436 + +(18, 2) + + + +5328310920313589898->-6847273675040002436 + + + + + +-6847273675040002436->2268567409594649842 + + + + + +-6847274936232341063 + +(19, 2) + + + +-5271230444997454145->-6847274936232341063 + + + + + +-6847274936232341063->2268567409594649842 + + + + + +-6847276197429339886 + +(20, 2) + + + +1868907299547856544->-6847276197429339886 + + + + + +-6847276197429339886->2268567409594649842 + + + + + +-6847277458630998905 + +(21, 2) + + + +8622045128323449149->-6847277458630998905 + + + + + +-6847277458630998905->2268567409594649842 + + + + + +-6847278719827997728 + +(22, 2) + + + +7655488486947663294->-6847278719827997728 + + + + + +-6847278719827997728->2268567409594649842 + + + + + +-6847279981020336355 + +(23, 2) + + + +4434248937406254995->-6847279981020336355 + + + + + +-6847279981020336355->2268567409594649842 + + + + + +-6847261063065354010 + +(24, 2) + + + +2998878464120799540->-6847261063065354010 + + + + + +-6847261063065354010->2268567409594649842 + + + + + +-6847262324267013029 + +(25, 2) + + + +-7651133118211551743->-6847262324267013029 + + + + + +-6847262324267013029->2268567409594649842 + + + + + +-6847263585464011852 + +(26, 2) + + + +-796067390339340926->-6847263585464011852 + + + + + +-6847263585464011852->2268567409594649842 + + + + + +-6847264846656350479 + +(27, 2) + + + +6755978969659815191->-6847264846656350479 + + + + + +-6847264846656350479->2268567409594649842 + + + + + +-6847266107853349302 + +(28, 2) + + + +2895657186813456536->-6847266107853349302 + + + + + +-6847266107853349302->2268567409594649842 + + + + + +-6847267369055008321 + +(29, 2) + + + +2325037936148553525->-6847267369055008321 + + + + + +-6847267369055008321->2268567409594649842 + + + + + +-6847268630252007144 + +(30, 2) + + + +5726971721246647990->-6847268630252007144 + + + + + +-6847268630252007144->2268567409594649842 + + + + + +-6847269891444345771 + +(31, 2) + + + +-3769857899901800053->-6847269891444345771 + + + + + +-6847269891444345771->2268567409594649842 + + + + + +-6847250973489363426 + +(32, 2) + + + +7405038051281652268->-6847250973489363426 + + + + + +-6847250973489363426->2268567409594649842 + + + + + +-6847252234691022445 + +(33, 2) + + + +-3319229209619479815->-6847252234691022445 + + + + + +-6847252234691022445->2268567409594649842 + + + + + +-6847253495888021268 + +(34, 2) + + + +-9218585940201761830->-6847253495888021268 + + + + + +-6847253495888021268->2268567409594649842 + + + + + +-6847254757080359895 + +(35, 2) + + + +-3674168660788982257->-6847254757080359895 + + + + + +-6847254757080359895->2268567409594649842 + + + + + +-6847256018277358718 + +(36, 2) + + + +3187748094154717488->-6847256018277358718 + + + + + +-6847256018277358718->2268567409594649842 + + + + + +-6847257279479017737 + +(37, 2) + + + +2852155922838332621->-6847257279479017737 + + + + + +-6847257279479017737->2268567409594649842 + + + + + +-6847258540676016560 + +(38, 2) + + + +685614591244829966->-6847258540676016560 + + + + + +-6847258540676016560->2268567409594649842 + + + + + +-6847259801868355187 + +(39, 2) + + + +5814156086091674339->-6847259801868355187 + + + + + +-6847259801868355187->2268567409594649842 + + + + + +-6847240883913372842 + +(40, 2) + + + +6085564958226103236->-6847240883913372842 + + + + + +-6847240883913372842->2268567409594649842 + + + + + +-6847242145115031861 + +(41, 2) + + + +3245989175914275153->-6847242145115031861 + + + + + +-6847242145115031861->2268567409594649842 + + + + + +-6847243406312030684 + +(42, 2) + + + +-5817998570605273902->-6847243406312030684 + + + + + +-6847243406312030684->2268567409594649842 + + + + + +-6847244667504369311 + +(43, 2) + + + +711075339292303271->-6847244667504369311 + + + + + +-6847244667504369311->2268567409594649842 + + + + + +-6847245928701368134 + +(44, 2) + + + +-1683731778835104984->-6847245928701368134 + + + + + +-6847245928701368134->2268567409594649842 + + + + + +-6847247189903027153 + +(45, 2) + + + +4137386294721355205->-6847247189903027153 + + + + + +-6847247189903027153->2268567409594649842 + + + + + +-6847248451100025976 + +(46, 2) + + + +975238977993362950->-6847248451100025976 + + + + + +-6847248451100025976->2268567409594649842 + + + + + +-6847249712292364603 + +(47, 2) + + + +-8521590608791411493->-6847249712292364603 + + + + + +-6847249712292364603->2268567409594649842 + + + + + +-6847230794337382258 + +(48, 2) + + + +2788302523013543100->-6847230794337382258 + + + + + +-6847230794337382258->2268567409594649842 + + + + + +-6847291331794490811 + +(0, 3) + + + +8995032920700762459->-6847291331794490811 + + + + + +3369230225407514717 + +simple-shuffle-combine + + + +-6847291331794490811->3369230225407514717 + + + + + +-6847292592993819732 + +(1, 3) + + + +4680587689285289626->-6847292592993819732 + + + + + +-6847292592993819732->3369230225407514717 + + + + + +-6847293854193148653 + +(2, 3) + + + +-4274668558010425479->-6847293854193148653 + + + + + +-6847293854193148653->3369230225407514717 + + + + + +-6847295115383157182 + +(3, 3) + + + +8308860109938309360->-6847295115383157182 + + + + + +-6847295115383157182->3369230225407514717 + + + + + +-6847296376582486103 + +(4, 3) + + + +2708015591099229839->-6847296376582486103 + + + + + +-6847296376582486103->3369230225407514717 + + + + + +-6847297637781815024 + +(5, 3) + + + +5666342200717070286->-6847297637781815024 + + + + + +-6847297637781815024->3369230225407514717 + + + + + +-6847298898981143945 + +(6, 3) + + + +-2063448707940247219->-6847298898981143945 + + + + + +-6847298898981143945->3369230225407514717 + + + + + +-6847300160171152474 + +(7, 3) + + + +300819336505097204->-6847300160171152474 + + + + + +-6847300160171152474->3369230225407514717 + + + + + +-6847281242218500227 + +(8, 3) + + + +-5115682359529666829->-6847281242218500227 + + + + + +-6847281242218500227->3369230225407514717 + + + + + +-6847282503417829148 + +(9, 3) + + + +8056059423068391570->-6847282503417829148 + + + + + +-6847282503417829148->3369230225407514717 + + + + + +-6847283764617158069 + +(10, 3) + + + +5011358343687835089->-6847283764617158069 + + + + + +-6847283764617158069->3369230225407514717 + + + + + +-6847285025807166598 + +(11, 3) + + + +-8599211999760001304->-6847285025807166598 + + + + + +-6847285025807166598->3369230225407514717 + + + + + +-6847286287006495519 + +(12, 3) + + + +8087417423913709607->-6847286287006495519 + + + + + +-6847286287006495519->3369230225407514717 + + + + + +-6847287548205824440 + +(13, 3) + + + +-5997990422892518970->-6847287548205824440 + + + + + +-6847287548205824440->3369230225407514717 + + + + + +-6847288809405153361 + +(14, 3) + + + +-5429702935223673275->-6847288809405153361 + + + + + +-6847288809405153361->3369230225407514717 + + + + + +-6847290070595161890 + +(15, 3) + + + +-1845951633683128852->-6847290070595161890 + + + + + +-6847290070595161890->3369230225407514717 + + + + + +-6847271152642509643 + +(16, 3) + + + +-5922413335747398677->-6847271152642509643 + + + + + +-6847271152642509643->3369230225407514717 + + + + + +-6847272413841838564 + +(17, 3) + + + +8583405443493537066->-6847272413841838564 + + + + + +-6847272413841838564->3369230225407514717 + + + + + +-6847273675041167485 + +(18, 3) + + + +6373097327572112073->-6847273675041167485 + + + + + +-6847273675041167485->3369230225407514717 + + + + + +-6847274936231176014 + +(19, 3) + + + +-8459440716446966720->-6847274936231176014 + + + + + +-6847274936231176014->3369230225407514717 + + + + + +-6847276197430504935 + +(20, 3) + + + +-4922843520047418081->-6847276197430504935 + + + + + +-6847276197430504935->3369230225407514717 + + + + + +-6847277458629833856 + +(21, 3) + + + +-3103904467101723042->-6847277458629833856 + + + + + +-6847277458629833856->3369230225407514717 + + + + + +-6847278719829162777 + +(22, 3) + + + +-1045986255288955747->-6847278719829162777 + + + + + +-6847278719829162777->3369230225407514717 + + + + + +-6847279981019171306 + +(23, 3) + + + +2813355614002150020->-6847279981019171306 + + + + + +-6847279981019171306->3369230225407514717 + + + + + +-6847261063066519059 + +(24, 3) + + + +-5010544406200074685->-6847261063066519059 + + + + + +-6847261063066519059->3369230225407514717 + + + + + +-6847262324265847980 + +(25, 3) + + + +-8703366091727748318->-6847262324265847980 + + + + + +-6847262324265847980->3369230225407514717 + + + + + +-6847263585465176901 + +(26, 3) + + + +259621023484209249->-6847263585465176901 + + + + + +-6847263585465176901->3369230225407514717 + + + + + +-6847264846655185430 + +(27, 3) + + + +5086452695504693816->-6847264846655185430 + + + + + +-6847264846655185430->3369230225407514717 + + + + + +-6847266107854514351 + +(28, 3) + + + +5697941532884840311->-6847266107854514351 + + + + + +-6847266107854514351->3369230225407514717 + + + + + +-6847267369053843272 + +(29, 3) + + + +775062942307134550->-6847267369053843272 + + + + + +-6847267369053843272->3369230225407514717 + + + + + +-6847268630253172193 + +(30, 3) + + + +6853617612633974165->-6847268630253172193 + + + + + +-6847268630253172193->3369230225407514717 + + + + + +-6847269891443180722 + +(31, 3) + + + +-6866774455711025028->-6847269891443180722 + + + + + +-6847269891443180722->3369230225407514717 + + + + + +-6847250973490528475 + +(32, 3) + + + +-419709346855106757->-6847250973490528475 + + + + + +-6847250973490528475->3369230225407514717 + + + + + +-6847252234689857396 + +(33, 3) + + + +5381122189635042426->-6847252234689857396 + + + + + +-6847252234689857396->3369230225407514717 + + + + + +-6847253495889186317 + +(34, 3) + + + +598742968415725145->-6847253495889186317 + + + + + +-6847253495889186317->3369230225407514717 + + + + + +-6847254757079194846 + +(35, 3) + + + +3110641678944335568->-6847254757079194846 + + + + + +-6847254757079194846->3369230225407514717 + + + + + +-6847256018278523767 + +(36, 3) + + + +-4803142261890561937->-6847256018278523767 + + + + + +-6847256018278523767->3369230225407514717 + + + + + +-6847257279477852688 + +(37, 3) + + + +-8163192485919267154->-6847257279477852688 + + + + + +-6847257279477852688->3369230225407514717 + + + + + +-6847258540677181609 + +(38, 3) + + + +2858287509046781741->-6847258540677181609 + + + + + +-6847258540677181609->3369230225407514717 + + + + + +-6847259801867190138 + +(39, 3) + + + +5436733318037790164->-6847259801867190138 + + + + + +-6847259801867190138->3369230225407514717 + + + + + +-6847240883914537891 + +(40, 3) + + + +-1429137792526549805->-6847240883914537891 + + + + + +-6847240883914537891->3369230225407514717 + + + + + +-6847242145113866812 + +(41, 3) + + + +3392862873804772978->-6847242145113866812 + + + + + +-6847242145113866812->3369230225407514717 + + + + + +-6847243406313195733 + +(42, 3) + + + +-6034700103474766927->-6847243406313195733 + + + + + +-6847243406313195733->3369230225407514717 + + + + + +-6847244667503204262 + +(43, 3) + + + +-1251840977629742904->-6847244667503204262 + + + + + +-6847244667503204262->3369230225407514717 + + + + + +-6847245928702533183 + +(44, 3) + + + +-1388430963211388409->-6847245928702533183 + + + + + +-6847245928702533183->3369230225407514717 + + + + + +-6847247189901862104 + +(45, 3) + + + +3164926128515845030->-6847247189901862104 + + + + + +-6847247189901862104->3369230225407514717 + + + + + +-6847248451101191025 + +(46, 3) + + + +2018630839030898725->-6847248451101191025 + + + + + +-6847248451101191025->3369230225407514717 + + + + + +-6847249712291199554 + +(47, 3) + + + +-1756452960081826868->-6847249712291199554 + + + + + +-6847249712291199554->3369230225407514717 + + + + + +-6847230794338547307 + +(48, 3) + + + +2506069369240048075->-6847230794338547307 + + + + + +-6847230794338547307->3369230225407514717 + + + + + +-6847291331790995664 + +(0, 4) + + + +-4119180910339601682->-6847291331790995664 + + + + + +4539803893240303192 + +simple-shuffle-combine + + + +-6847291331790995664->4539803893240303192 + + + + + +-6847292592987994487 + +(1, 4) + + + +6560038413356320367->-6847292592987994487 + + + + + +-6847292592987994487->4539803893240303192 + + + + + +-6847293854184993310 + +(2, 4) + + + +3397640077024762512->-6847293854184993310 + + + + + +-6847293854184993310->4539803893240303192 + + + + + +-6847295115391312525 + +(3, 4) + + + +-423330542066881831->-6847295115391312525 + + + + + +-6847295115391312525->4539803893240303192 + + + + + +-6847296376588311348 + +(4, 4) + + + +7004998100991881914->-6847296376588311348 + + + + + +-6847296376588311348->4539803893240303192 + + + + + +-6847297637785310171 + +(5, 4) + + + +1954907472151383611->-6847297637785310171 + + + + + +-6847297637785310171->4539803893240303192 + + + + + +-6847298898982308994 + +(6, 4) + + + +7608553450364041356->-6847298898982308994 + + + + + +-6847298898982308994->4539803893240303192 + + + + + +-6847300160169987425 + +(7, 4) + + + +7650154878988875029->-6847300160169987425 + + + + + +-6847300160169987425->4539803893240303192 + + + + + +-6847281242215005080 + +(8, 4) + + + +7668218353096620262->-6847281242215005080 + + + + + +-6847281242215005080->4539803893240303192 + + + + + +-6847282503412003903 + +(9, 4) + + + +-7510353515772233721->-6847282503412003903 + + + + + +-6847282503412003903->4539803893240303192 + + + + + +-6847283764609002726 + +(10, 4) + + + +6119838159620707208->-6847283764609002726 + + + + + +-6847283764609002726->4539803893240303192 + + + + + +-6847285025815321941 + +(11, 4) + + + +1057864508383055921->-6847285025815321941 + + + + + +-6847285025815321941->4539803893240303192 + + + + + +-6847286287012320764 + +(12, 4) + + + +-970229545114285134->-6847286287012320764 + + + + + +-6847286287012320764->4539803893240303192 + + + + + +-6847287548209319587 + +(13, 4) + + + +-1711822607036620845->-6847287548209319587 + + + + + +-6847287548209319587->4539803893240303192 + + + + + +-6847288809406318410 + +(14, 4) + + + +-5497757209277550300->-6847288809406318410 + + + + + +-6847288809406318410->4539803893240303192 + + + + + +-6847290070593996841 + +(15, 4) + + + +-1703506801389939027->-6847290070593996841 + + + + + +-6847290070593996841->4539803893240303192 + + + + + +-6847271152639014496 + +(16, 4) + + + +-1570576633620998786->-6847271152639014496 + + + + + +-6847271152639014496->4539803893240303192 + + + + + +-6847272413836013319 + +(17, 4) + + + +-7097949457268237825->-6847272413836013319 + + + + + +-6847272413836013319->4539803893240303192 + + + + + +-6847273675033012142 + +(18, 4) + + + +6330275645409165792->-6847273675033012142 + + + + + +-6847273675033012142->4539803893240303192 + + + + + +-6847274936239331357 + +(19, 4) + + + +3765743178776105->-6847274936239331357 + + + + + +-6847274936239331357->4539803893240303192 + + + + + +-6847276197436330180 + +(20, 4) + + + +-694676843814286006->-6847276197436330180 + + + + + +-6847276197436330180->4539803893240303192 + + + + + +-6847277458633329003 + +(21, 4) + + + +-6898587583792977717->-6847277458633329003 + + + + + +-6847277458633329003->4539803893240303192 + + + + + +-6847278719830327826 + +(22, 4) + + + +521722753064620828->-6847278719830327826 + + + + + +-6847278719830327826->4539803893240303192 + + + + + +-6847279981018006257 + +(23, 4) + + + +-6931896946889560155->-6847279981018006257 + + + + + +-6847279981018006257->4539803893240303192 + + + + + +-6847261063063023912 + +(24, 4) + + + +-8985553183631030410->-6847261063063023912 + + + + + +-6847261063063023912->4539803893240303192 + + + + + +-6847262324260022735 + +(25, 4) + + + +995378323287150423->-6847262324260022735 + + + + + +-6847262324260022735->4539803893240303192 + + + + + +-6847263585457021558 + +(26, 4) + + + +8920021036213978840->-6847263585457021558 + + + + + +-6847263585457021558->4539803893240303192 + + + + + +-6847264846663340773 + +(27, 4) + + + +-2512961549035590975->-6847264846663340773 + + + + + +-6847264846663340773->4539803893240303192 + + + + + +-6847266107860339596 + +(28, 4) + + + +-4550218554994135230->-6847266107860339596 + + + + + +-6847266107860339596->4539803893240303192 + + + + + +-6847267369057338419 + +(29, 4) + + + +5357941405872362275->-6847267369057338419 + + + + + +-6847267369057338419->4539803893240303192 + + + + + +-6847268630254337242 + +(30, 4) + + + +-8936168839752481676->-6847268630254337242 + + + + + +-6847268630254337242->4539803893240303192 + + + + + +-6847269891442015673 + +(31, 4) + + + +-7941325392630284803->-6847269891442015673 + + + + + +-6847269891442015673->4539803893240303192 + + + + + +-6847250973487033328 + +(32, 4) + + + +-4509427441088549682->-6847250973487033328 + + + + + +-6847250973487033328->4539803893240303192 + + + + + +-6847252234684032151 + +(33, 4) + + + +-1734684433498870449->-6847252234684032151 + + + + + +-6847252234684032151->4539803893240303192 + + + + + +-6847253495881030974 + +(34, 4) + + + +8344565182692097136->-6847253495881030974 + + + + + +-6847253495881030974->4539803893240303192 + + + + + +-6847254757087350189 + +(35, 4) + + + +-5697043306872132423->-6847254757087350189 + + + + + +-6847254757087350189->4539803893240303192 + + + + + +-6847256018284349012 + +(36, 4) + + + +1949869690151030938->-6847256018284349012 + + + + + +-6847256018284349012->4539803893240303192 + + + + + +-6847257279481347835 + +(37, 4) + + + +-4135808365496109029->-6847257279481347835 + + + + + +-6847257279481347835->4539803893240303192 + + + + + +-6847258540678346658 + +(38, 4) + + + +-6238596950200801684->-6847258540678346658 + + + + + +-6847258540678346658->4539803893240303192 + + + + + +-6847259801866025089 + +(39, 4) + + + +4388625239350553589->-6847259801866025089 + + + + + +-6847259801866025089->4539803893240303192 + + + + + +-6847240883911042744 + +(40, 4) + + + +-5451825285884835130->-6847240883911042744 + + + + + +-6847240883911042744->4539803893240303192 + + + + + +-6847242145108041567 + +(41, 4) + + + +-5490628949868201497->-6847242145108041567 + + + + + +-6847242145108041567->4539803893240303192 + + + + + +-6847243406305040390 + +(42, 4) + + + +3907789468740405864->-6847243406305040390 + + + + + +-6847243406305040390->4539803893240303192 + + + + + +-6847244667511359605 + +(43, 4) + + + +8309773607733845521->-6847244667511359605 + + + + + +-6847244667511359605->4539803893240303192 + + + + + +-6847245928708358428 + +(44, 4) + + + +7546014880032965266->-6847245928708358428 + + + + + +-6847245928708358428->4539803893240303192 + + + + + +-6847247189905357251 + +(45, 4) + + + +7152540080132485555->-6847247189905357251 + + + + + +-6847247189905357251->4539803893240303192 + + + + + +-6847248451102356074 + +(46, 4) + + + +5199941565409328900->-6847248451102356074 + + + + + +-6847248451102356074->4539803893240303192 + + + + + +-6847249712290034505 + +(47, 4) + + + +5753685901314578573->-6847249712290034505 + + + + + +-6847249712290034505->4539803893240303192 + + + + + +-6847230794335052160 + +(48, 4) + + + +-371220232816908450->-6847230794335052160 + + + + + +-6847230794335052160->4539803893240303192 + + + + + +-6847291331792160713 + +(0, 5) + + + +7611195280567332109->-6847291331792160713 + + + + + +6082374622813686467 + +simple-shuffle-combine + + + +-6847291331792160713->6082374622813686467 + + + + + +-6847292592986829438 + +(1, 5) + + + +5516664421532978992->-6847292592986829438 + + + + + +-6847292592986829438->6082374622813686467 + + + + + +-6847293854186158359 + +(2, 5) + + + +4370047218869479887->-6847293854186158359 + + + + + +-6847293854186158359->6082374622813686467 + + + + + +-6847295115390147476 + +(3, 5) + + + +8133971699876450394->-6847295115390147476 + + + + + +-6847295115390147476->6082374622813686467 + + + + + +-6847296376589476397 + +(4, 5) + + + +-1406977365211904711->-6847296376589476397 + + + + + +-6847296376589476397->6082374622813686467 + + + + + +-6847297637784145122 + +(5, 5) + + + +-1170154662713482964->-6847297637784145122 + + + + + +-6847297637784145122->6082374622813686467 + + + + + +-6847298898983474043 + +(6, 5) + + + +971256081963288731->-6847298898983474043 + + + + + +-6847298898983474043->6082374622813686467 + + + + + +-6847300160168822376 + +(7, 5) + + + +8438954423858137654->-6847300160168822376 + + + + + +-6847300160168822376->6082374622813686467 + + + + + +-6847281242216170129 + +(8, 5) + + + +829811772141247237->-6847281242216170129 + + + + + +-6847281242216170129->6082374622813686467 + + + + + +-6847282503410838854 + +(9, 5) + + + +-8492755242329198296->-6847282503410838854 + + + + + +-6847282503410838854->6082374622813686467 + + + + + +-6847283764610167775 + +(10, 5) + + + +7172107410091169383->-6847283764610167775 + + + + + +-6847283764610167775->6082374622813686467 + + + + + +-6847285025814156892 + +(11, 5) + + + +14490516525073746->-6847285025814156892 + + + + + +-6847285025814156892->6082374622813686467 + + + + + +-6847286287013485813 + +(12, 5) + + + +-71109137058747759->-6847286287013485813 + + + + + +-6847286287013485813->6082374622813686467 + + + + + +-6847287548208154538 + +(13, 5) + + + +-3453392277428092220->-6847287548208154538 + + + + + +-6847287548208154538->6082374622813686467 + + + + + +-6847288809407483459 + +(14, 5) + + + +-2840795081896510925->-6847288809407483459 + + + + + +-6847288809407483459->6082374622813686467 + + + + + +-6847290070592831792 + +(15, 5) + + + +-4820836271751174002->-6847290070592831792 + + + + + +-6847290070592831792->6082374622813686467 + + + + + +-6847271152640179545 + +(16, 5) + + + +-30567807916130211->-6847271152640179545 + + + + + +-6847271152640179545->6082374622813686467 + + + + + +-6847272413834848270 + +(17, 5) + + + +1528145207011666816->-6847272413834848270 + + + + + +-6847272413834848270->6082374622813686467 + + + + + +-6847273675034177191 + +(18, 5) + + + +6520838046366307423->-6847273675034177191 + + + + + +-6847273675034177191->6082374622813686467 + + + + + +-6847274936238166308 + +(19, 5) + + + +-969781542124270870->-6847274936238166308 + + + + + +-6847274936238166308->6082374622813686467 + + + + + +-6847276197437495229 + +(20, 5) + + + +427517742427589769->-6847276197437495229 + + + + + +-6847276197437495229->6082374622813686467 + + + + + +-6847277458632163954 + +(21, 5) + + + +-51696539686298692->-6847277458632163954 + + + + + +-6847277458632163954->6082374622813686467 + + + + + +-6847278719831492875 + +(22, 5) + + + +-7956893847309398997->-6847278719831492875 + + + + + +-6847278719831492875->6082374622813686467 + + + + + +-6847279981016841208 + +(23, 5) + + + +967223098131784070->-6847279981016841208 + + + + + +-6847279981016841208->6082374622813686467 + + + + + +-6847261063064188961 + +(24, 5) + + + +2022009538427194965->-6847261063064188961 + + + + + +-6847261063064188961->6082374622813686467 + + + + + +-6847262324258857686 + +(25, 5) + + + +1138798804532528248->-6847262324258857686 + + + + + +-6847262324258857686->6082374622813686467 + + + + + +-6847263585458186607 + +(26, 5) + + + +-8483344498547619401->-6847263585458186607 + + + + + +-6847263585458186607->6082374622813686467 + + + + + +-6847264846662175724 + +(27, 5) + + + +-3566302836380318750->-6847264846662175724 + + + + + +-6847264846662175724->6082374622813686467 + + + + + +-6847266107861504645 + +(28, 5) + + + +-5251222663889842655->-6847266107861504645 + + + + + +-6847266107861504645->6082374622813686467 + + + + + +-6847267369056173370 + +(29, 5) + + + +3368411928006535700->-6847267369056173370 + + + + + +-6847267369056173370->6082374622813686467 + + + + + +-6847268630255502291 + +(30, 5) + + + +-7421663438984818301->-6847268630255502291 + + + + + +-6847268630255502291->6082374622813686467 + + + + + +-6847269891440850624 + +(31, 5) + + + +-1165387318920932578->-6847269891440850624 + + + + + +-6847269891440850624->6082374622813686467 + + + + + +-6847250973488198377 + +(32, 5) + + + +-3601393856466732307->-6847250973488198377 + + + + + +-6847250973488198377->6082374622813686467 + + + + + +-6847252234682867102 + +(33, 5) + + + +5561433954438658576->-6847252234682867102 + + + + + +-6847252234682867102->6082374622813686467 + + + + + +-6847253495882196023 + +(34, 5) + + + +-8644324518070401105->-6847253495882196023 + + + + + +-6847253495882196023->6082374622813686467 + + + + + +-6847254757086185140 + +(35, 5) + + + +1814203912623746618->-6847254757086185140 + + + + + +-6847254757086185140->6082374622813686467 + + + + + +-6847256018285514061 + +(36, 5) + + + +5137007951330412313->-6847256018285514061 + + + + + +-6847256018285514061->6082374622813686467 + + + + + +-6847257279480182786 + +(37, 5) + + + +3825666883521373196->-6847257279480182786 + + + + + +-6847257279480182786->6082374622813686467 + + + + + +-6847258540679511707 + +(38, 5) + + + +-5265049701461119109->-6847258540679511707 + + + + + +-6847258540679511707->6082374622813686467 + + + + + +-6847259801864860040 + +(39, 5) + + + +-7245085214589322986->-6847259801864860040 + + + + + +-6847259801864860040->6082374622813686467 + + + + + +-6847240883912207793 + +(40, 5) + + + +-4408433460319478555->-6847240883912207793 + + + + + +-6847240883912207793->6082374622813686467 + + + + + +-6847242145106876518 + +(41, 5) + + + +-6467496822481486072->-6847242145106876518 + + + + + +-6847242145106876518->6082374622813686467 + + + + + +-6847243406306205439 + +(42, 5) + + + +7080375167217210439->-6847243406306205439 + + + + + +-6847243406306205439->6082374622813686467 + + + + + +-6847244667510194556 + +(43, 5) + + + +7265291214475389746->-6847244667510194556 + + + + + +-6847244667510194556->6082374622813686467 + + + + + +-6847245928709523477 + +(44, 5) + + + +7877615050603446641->-6847245928709523477 + + + + + +-6847245928709523477->6082374622813686467 + + + + + +-6847247189904192202 + +(45, 5) + + + +5631832359963657380->-6847247189904192202 + + + + + +-6847247189904192202->6082374622813686467 + + + + + +-6847248451103521123 + +(46, 5) + + + +6243283774177427475->-6847248451103521123 + + + + + +-6847248451103521123->6082374622813686467 + + + + + +-6847249712288869456 + +(47, 5) + + + +4188986975589836398->-6847249712288869456 + + + + + +-6847249712288869456->6082374622813686467 + + + + + +-6847230794336217209 + +(48, 5) + + + +-7881337794243569091->-6847230794336217209 + + + + + +-6847230794336217209->6082374622813686467 + + + + + +-6847291331788665566 + +(0, 6) + + + +1879494051309515984->-6847291331788665566 + + + + + +7134231313787865742 + +simple-shuffle-combine + + + +-6847291331788665566->7134231313787865742 + + + + + +-6847292592990324585 + +(1, 6) + + + +-8845895420737534099->-6847292592990324585 + + + + + +-6847292592990324585->7134231313787865742 + + + + + +-6847293854187323408 + +(2, 6) + + + +-1355645835306697554->-6847293854187323408 + + + + + +-6847293854187323408->7134231313787865742 + + + + + +-6847295115388982427 + +(3, 6) + + + +7583795547091185019->-6847295115388982427 + + + + + +-6847295115388982427->7134231313787865742 + + + + + +-6847296376585981250 + +(4, 6) + + + +-4361143243973501236->-6847296376585981250 + + + + + +-6847296376585981250->7134231313787865742 + + + + + +-6847297637787640269 + +(5, 6) + + + +4456096072712817561->-6847297637787640269 + + + + + +-6847297637787640269->7134231313787865742 + + + + + +-6847298898984639092 + +(6, 6) + + + +754554506970582906->-6847298898984639092 + + + + + +-6847298898984639092->7134231313787865742 + + + + + +-6847300160167657327 + +(7, 6) + + + +7364439732854986679->-6847300160167657327 + + + + + +-6847300160167657327->7134231313787865742 + + + + + +-6847281242212674982 + +(8, 6) + + + +6832686957933287112->-6847281242212674982 + + + + + +-6847281242212674982->7134231313787865742 + + + + + +-6847282503414334001 + +(9, 6) + + + +-4438895229115951771->-6847282503414334001 + + + + + +-6847282503414334001->7134231313787865742 + + + + + +-6847283764611332824 + +(10, 6) + + + +-883174416731428442->-6847283764611332824 + + + + + +-6847283764611332824->7134231313787865742 + + + + + +-6847285025812991843 + +(11, 6) + + + +-4301887946267048429->-6847285025812991843 + + + + + +-6847285025812991843->7134231313787865742 + + + + + +-6847286287009990666 + +(12, 6) + + + +-5272894220001373084->-6847286287009990666 + + + + + +-6847286287009990666->7134231313787865742 + + + + + +-6847287548211649685 + +(13, 6) + + + +132329273046955505->-6847287548211649685 + + + + + +-6847287548211649685->7134231313787865742 + + + + + +-6847288809408648508 + +(14, 6) + + + +-1347494824610795150->-6847288809408648508 + + + + + +-6847288809408648508->7134231313787865742 + + + + + +-6847290070591666743 + +(15, 6) + + + +-6935742065034205777->-6847290070591666743 + + + + + +-6847290070591666743->7134231313787865742 + + + + + +-6847271152636684398 + +(16, 6) + + + +-2851604285814765536->-6847271152636684398 + + + + + +-6847271152636684398->7134231313787865742 + + + + + +-6847272413838343417 + +(17, 6) + + + +3359708357082430141->-6847272413838343417 + + + + + +-6847272413838343417->7134231313787865742 + + + + + +-6847273675035342240 + +(18, 6) + + + +-9007677172844180674->-6847273675035342240 + + + + + +-6847273675035342240->7134231313787865742 + + + + + +-6847274936237001259 + +(19, 6) + + + +-189823072113760245->-6847274936237001259 + + + + + +-6847274936237001259->7134231313787865742 + + + + + +-6847276197434000082 + +(20, 6) + + + +6308367514484586844->-6847276197434000082 + + + + + +-6847276197434000082->7134231313787865742 + + + + + +-6847277458635659101 + +(21, 6) + + + +6852480346339315433->-6847277458635659101 + + + + + +-6847277458635659101->7134231313787865742 + + + + + +-6847278719832657924 + +(22, 6) + + + +-6974487624653720822->-6847278719832657924 + + + + + +-6847278719832657924->7134231313787865742 + + + + + +-6847279981015676159 + +(23, 6) + + + +1114097933383668295->-6847279981015676159 + + + + + +-6847279981015676159->7134231313787865742 + + + + + +-6847261063060693814 + +(24, 6) + + + +-2056894835390313960->-6847261063060693814 + + + + + +-6847261063060693814->7134231313787865742 + + + + + +-6847262324262352833 + +(25, 6) + + + +6359619510700424373->-6847262324262352833 + + + + + +-6847262324262352833->7134231313787865742 + + + + + +-6847263585459351656 + +(26, 6) + + + +-6417892052090158026->-6847263585459351656 + + + + + +-6847263585459351656->7134231313787865742 + + + + + +-6847264846661010675 + +(27, 6) + + + +-7872718535325869725->-6847264846661010675 + + + + + +-6847264846661010675->7134231313787865742 + + + + + +-6847266107858009498 + +(28, 6) + + + +-8792621140124309580->-6847266107858009498 + + + + + +-6847266107858009498->7134231313787865742 + + + + + +-6847267369059668517 + +(29, 6) + + + +6761025966748881025->-6847267369059668517 + + + + + +-6847267369059668517->7134231313787865742 + + + + + +-6847268630256667340 + +(30, 6) + + + +-6377181124395619326->-6847268630256667340 + + + + + +-6847268630256667340->7134231313787865742 + + + + + +-6847269891439685575 + +(31, 6) + + + +7526105108686566463->-6847269891439685575 + + + + + +-6847269891439685575->7134231313787865742 + + + + + +-6847250973484703230 + +(32, 6) + + + +2362553733096571568->-6847250973484703230 + + + + + +-6847250973484703230->7134231313787865742 + + + + + +-6847252234686362249 + +(33, 6) + + + +-1541077757860184499->-6847252234686362249 + + + + + +-6847252234686362249->7134231313787865742 + + + + + +-6847253495883361072 + +(34, 6) + + + +1323802984103596686->-6847253495883361072 + + + + + +-6847253495883361072->7134231313787865742 + + + + + +-6847254757085020091 + +(35, 6) + + + +2571885246635588443->-6847254757085020091 + + + + + +-6847254757085020091->7134231313787865742 + + + + + +-6847256018282018914 + +(36, 6) + + + +8927319731613346988->-6847256018282018914 + + + + + +-6847256018282018914->7134231313787865742 + + + + + +-6847257279483677933 + +(37, 6) + + + +-38731361119123079->-6847257279483677933 + + + + + +-6847257279483677933->7134231313787865742 + + + + + +-6847258540680676756 + +(38, 6) + + + +6394147626301528666->-6847258540680676756 + + + + + +-6847258540680676756->7134231313787865742 + + + + + +-6847259801863694991 + +(39, 6) + + + +1224672456174284439->-6847259801863694991 + + + + + +-6847259801863694991->7134231313787865742 + + + + + +-6847240883908712646 + +(40, 6) + + + +8857944242671264936->-6847240883908712646 + + + + + +-6847240883908712646->7134231313787865742 + + + + + +-6847242145110371665 + +(41, 6) + + + +-74176913691740347->-6847242145110371665 + + + + + +-6847242145110371665->7134231313787865742 + + + + + +-6847243406307370488 + +(42, 6) + + + +8043955121561332614->-6847243406307370488 + + + + + +-6847243406307370488->7134231313787865742 + + + + + +-6847244667509029507 + +(43, 6) + + + +7456220286210633971->-6847244667509029507 + + + + + +-6847244667509029507->7134231313787865742 + + + + + +-6847245928706028330 + +(44, 6) + + + +3744679640852546116->-6847245928706028330 + + + + + +-6847245928706028330->7134231313787865742 + + + + + +-6847247189907687349 + +(45, 6) + + + +-8604301462073124911->-6847247189907687349 + + + + + +-6847247189907687349->7134231313787865742 + + + + + +-6847248451104686172 + +(46, 6) + + + +8946100326454656850->-6847248451104686172 + + + + + +-6847248451104686172->7134231313787865742 + + + + + +-6847249712287704407 + +(47, 6) + + + +-6899526248249210993->-6847249712287704407 + + + + + +-6847249712287704407->7134231313787865742 + + + + + +-6847230794332722062 + +(48, 6) + + + +-1682464094607699200->-6847230794332722062 + + + + + +-6847230794332722062->7134231313787865742 + + + + + +-6847291331789830615 + +(0, 7) + + + +4480203053389087759->-6847291331789830615 + + + + + +8234894118550315417 + +simple-shuffle-combine + + + +-6847291331789830615->8234894118550315417 + + + + + +-6847292592989159536 + +(1, 7) + + + +-375185646789188274->-6847292592989159536 + + + + + +-6847292592989159536->8234894118550315417 + + + + + +-6847293854188488457 + +(2, 7) + + + +8616932943226245837->-6847293854188488457 + + + + + +-6847293854188488457->8234894118550315417 + + + + + +-6847295115387817378 + +(3, 7) + + + +-2171456135574231956->-6847295115387817378 + + + + + +-6847295115387817378->8234894118550315417 + + + + + +-6847296376587146299 + +(4, 7) + + + +5465043459040741339->-6847296376587146299 + + + + + +-6847296376587146299->8234894118550315417 + + + + + +-6847297637786475220 + +(5, 7) + + + +1809137598641736986->-6847297637786475220 + + + + + +-6847297637786475220->8234894118550315417 + + + + + +-6847298898985804141 + +(6, 7) + + + +2337344029002597881->-6847298898985804141 + + + + + +-6847298898985804141->8234894118550315417 + + + + + +-6847300160166492278 + +(7, 7) + + + +-5258933739439424296->-6847300160166492278 + + + + + +-6847300160166492278->8234894118550315417 + + + + + +-6847281242213840031 + +(8, 7) + + + +7886023711490110887->-6847281242213840031 + + + + + +-6847281242213840031->8234894118550315417 + + + + + +-6847282503413168952 + +(9, 7) + + + +4160472001651424070->-6847282503413168952 + + + + + +-6847282503413168952->8234894118550315417 + + + + + +-6847283764612497873 + +(10, 7) + + + +8963094060282452933->-6847283764612497873 + + + + + +-6847283764612497873->8234894118550315417 + + + + + +-6847285025811826794 + +(11, 7) + + + +-5349728331088241404->-6847285025811826794 + + + + + +-6847285025811826794->8234894118550315417 + + + + + +-6847286287011155715 + +(12, 7) + + + +6460919666168667507->-6847286287011155715 + + + + + +-6847286287011155715->8234894118550315417 + + + + + +-6847287548210484636 + +(13, 7) + + + +-363847941606469870->-6847287548210484636 + + + + + +-6847287548210484636->8234894118550315417 + + + + + +-6847288809409813557 + +(14, 7) + + + +138496045440159825->-6847288809409813557 + + + + + +-6847288809409813557->8234894118550315417 + + + + + +-6847290070590501694 + +(15, 7) + + + +-7431922684982193552->-6847290070590501694 + + + + + +-6847290070590501694->8234894118550315417 + + + + + +-6847271152637849447 + +(16, 7) + + + +6898072677899988639->-6847271152637849447 + + + + + +-6847271152637849447->8234894118550315417 + + + + + +-6847272413837178368 + +(17, 7) + + + +3505456989407769566->-6847272413837178368 + + + + + +-6847272413837178368->8234894118550315417 + + + + + +-6847273675036507289 + +(18, 7) + + + +9222386601420645917->-6847273675036507289 + + + + + +-6847273675036507289->8234894118550315417 + + + + + +-6847274936235836210 + +(19, 7) + + + +7168371417681878780->-6847274936235836210 + + + + + +-6847274936235836210->8234894118550315417 + + + + + +-6847276197435165131 + +(20, 7) + + + +7844907532395751019->-6847276197435165131 + + + + + +-6847276197435165131->8234894118550315417 + + + + + +-6847277458634494052 + +(21, 7) + + + +-5845246289797216342->-6847277458634494052 + + + + + +-6847277458634494052->8234894118550315417 + + + + + +-6847278719833822973 + +(22, 7) + + + +4733729393398606153->-6847278719833822973 + + + + + +-6847278719833822973->8234894118550315417 + + + + + +-6847279981014511110 + +(23, 7) + + + +-1699659099516841880->-6847279981014511110 + + + + + +-6847279981014511110->8234894118550315417 + + + + + +-6847261063061858863 + +(24, 7) + + + +-2202661268098578185->-6847261063061858863 + + + + + +-6847261063061858863->8234894118550315417 + + + + + +-6847262324261187784 + +(25, 7) + + + +4192485023931056598->-6847262324261187784 + + + + + +-6847262324261187784->8234894118550315417 + + + + + +-6847263585460516705 + +(26, 7) + + + +-4179822429687382251->-6847263585460516705 + + + + + +-6847263585460516705->8234894118550315417 + + + + + +-6847264846659845626 + +(27, 7) + + + +-8798483906242345900->-6847264846659845626 + + + + + +-6847264846659845626->8234894118550315417 + + + + + +-6847266107859174547 + +(28, 7) + + + +-5593592582358937405->-6847266107859174547 + + + + + +-6847266107859174547->8234894118550315417 + + + + + +-6847267369058503468 + +(29, 7) + + + +6330384880165906850->-6847267369058503468 + + + + + +-6847267369058503468->8234894118550315417 + + + + + +-6847268630257832389 + +(30, 7) + + + +-4883471413927768351->-6847268630257832389 + + + + + +-6847268630257832389->8234894118550315417 + + + + + +-6847269891438520526 + +(31, 7) + + + +5985139638180234688->-6847269891438520526 + + + + + +-6847269891438520526->8234894118550315417 + + + + + +-6847250973485868279 + +(32, 7) + + + +-5004499729506557457->-6847250973485868279 + + + + + +-6847250973485868279->8234894118550315417 + + + + + +-6847252234685197200 + +(33, 7) + + + +-2520046361722321874->-6847252234685197200 + + + + + +-6847252234685197200->8234894118550315417 + + + + + +-6847253495884526121 + +(34, 7) + + + +2306225949697138861->-6847253495884526121 + + + + + +-6847253495884526121->8234894118550315417 + + + + + +-6847254757083855042 + +(35, 7) + + + +1524081176457851468->-6847254757083855042 + + + + + +-6847254757083855042->8234894118550315417 + + + + + +-6847256018283183963 + +(36, 7) + + + +-314889215400903237->-6847256018283183963 + + + + + +-6847256018283183963->8234894118550315417 + + + + + +-6847257279482512884 + +(37, 7) + + + +6751911098090385146->-6847257279482512884 + + + + + +-6847257279482512884->8234894118550315417 + + + + + +-6847258540681841805 + +(38, 7) + + + +7364236061603278041->-6847258540681841805 + + + + + +-6847258540681841805->8234894118550315417 + + + + + +-6847259801862529942 + +(39, 7) + + + +-242040856342411336->-6847259801862529942 + + + + + +-6847259801862529942->8234894118550315417 + + + + + +-6847240883909877695 + +(40, 7) + + + +-8540995840655632505->-6847240883909877695 + + + + + +-6847240883909877695->8234894118550315417 + + + + + +-6847242145109206616 + +(41, 7) + + + +-1046600791857928922->-6847242145109206616 + + + + + +-6847242145109206616->8234894118550315417 + + + + + +-6847243406308535537 + +(42, 7) + + + +9088458790437050789->-6847243406308535537 + + + + + +-6847243406308535537->8234894118550315417 + + + + + +-6847244667507864458 + +(43, 7) + + + +3148696193676109796->-6847244667507864458 + + + + + +-6847244667507864458->8234894118550315417 + + + + + +-6847245928707193379 + +(44, 7) + + + +6472572301351742291->-6847245928707193379 + + + + + +-6847245928707193379->8234894118550315417 + + + + + +-6847247189906522300 + +(45, 7) + + + +8227087680380607730->-6847247189906522300 + + + + + +-6847247189906522300->8234894118550315417 + + + + + +-6847248451105851221 + +(46, 7) + + + +9099433518524248625->-6847248451105851221 + + + + + +-6847248451105851221->8234894118550315417 + + + + + +-6847249712286539358 + +(47, 7) + + + +1739889128364167248->-6847249712286539358 + + + + + +-6847249712286539358->8234894118550315417 + + + + + +-6847230794333887111 + +(48, 7) + + + +-708916802584337025->-6847230794333887111 + + + + + +-6847230794333887111->8234894118550315417 + + + + + +-6847291331804976252 + +(0, 8) + + + +2107302800818605618->-6847291331804976252 + + + + + +1493623177110034692 + +simple-shuffle-combine + + + +-6847291331804976252->1493623177110034692 + + + + + +-6847292593001975075 + +(1, 8) + + + +-5317347265349851565->-6847292593001975075 + + + + + +-6847292593001975075->1493623177110034692 + + + + + +-6847293854198973898 + +(2, 8) + + + +2328458508666252196->-6847293854198973898 + + + + + +-6847293854198973898->1493623177110034692 + + + + + +-6847295115395972721 + +(3, 8) + + + +7525656288220581669->-6847295115395972721 + + + + + +-6847295115395972721->1493623177110034692 + + + + + +-6847296376592971544 + +(4, 8) + + + +4306986957390682214->-6847296376592971544 + + + + + +-6847296376592971544->1493623177110034692 + + + + + +-6847297637789970367 + +(5, 8) + + + +6918549934797842311->-6847297637789970367 + + + + + +-6847297637789970367->1493623177110034692 + + + + + +-6847298898986969190 + +(6, 8) + + + +-5172758483369357560->-6847298898986969190 + + + + + +-6847298898986969190->1493623177110034692 + + + + + +-6847300160165327229 + +(7, 8) + + + +3218574475053200329->-6847300160165327229 + + + + + +-6847300160165327229->1493623177110034692 + + + + + +-6847281242210344884 + +(8, 8) + + + +3686465368105103162->-6847281242210344884 + + + + + +-6847281242210344884->1493623177110034692 + + + + + +-6847282503407343707 + +(9, 8) + + + +6457050598155703995->-6847282503407343707 + + + + + +-6847282503407343707->1493623177110034692 + + + + + +-6847283764604342530 + +(10, 8) + + + +-1885763688648309492->-6847283764604342530 + + + + + +-6847283764604342530->1493623177110034692 + + + + + +-6847285025801341353 + +(11, 8) + + + +1538001814289544237->-6847285025801341353 + + + + + +-6847285025801341353->1493623177110034692 + + + + + +-6847286286998340176 + +(12, 8) + + + +720494030768284782->-6847286286998340176 + + + + + +-6847286286998340176->1493623177110034692 + + + + + +-6847287548195338999 + +(13, 8) + + + +1039934370594930671->-6847287548195338999 + + + + + +-6847287548195338999->1493623177110034692 + + + + + +-6847288809392337822 + +(14, 8) + + + +-1062854249564620784->-6847288809392337822 + + + + + +-6847288809392337822->1493623177110034692 + + + + + +-6847290070607977429 + +(15, 8) + + + +-4951205630973585743->-6847290070607977429 + + + + + +-6847290070607977429->1493623177110034692 + + + + + +-6847271152652995084 + +(16, 8) + + + +-4326762512391319102->-6847271152652995084 + + + + + +-6847271152652995084->1493623177110034692 + + + + + +-6847272413849993907 + +(17, 8) + + + +-920222243153004125->-6847272413849993907 + + + + + +-6847272413849993907->1493623177110034692 + + + + + +-6847273675046992730 + +(18, 8) + + + +4531461257890406132->-6847273675046992730 + + + + + +-6847273675046992730->1493623177110034692 + + + + + +-6847274936243991553 + +(19, 8) + + + +-1655923121234460811->-6847274936243991553 + + + + + +-6847274936243991553->1493623177110034692 + + + + + +-6847276197440990376 + +(20, 8) + + + +7236301947436815094->-6847276197440990376 + + + + + +-6847276197440990376->1493623177110034692 + + + + + +-6847277458637989199 + +(21, 8) + + + +-2493638213323474217->-6847277458637989199 + + + + + +-6847277458637989199->1493623177110034692 + + + + + +-6847278719834988022 + +(22, 8) + + + +5706168329418263128->-6847278719834988022 + + + + + +-6847278719834988022->1493623177110034692 + + + + + +-6847279981013346061 + +(23, 8) + + + +5739543564280470361->-6847279981013346061 + + + + + +-6847279981013346061->1493623177110034692 + + + + + +-6847261063058363716 + +(24, 8) + + + +5060671597911452106->-6847261063058363716 + + + + + +-6847261063058363716->1493623177110034692 + + + + + +-6847262324255362539 + +(25, 8) + + + +-4047083629636983477->-6847262324255362539 + + + + + +-6847262324255362539->1493623177110034692 + + + + + +-6847263585452361362 + +(26, 8) + + + +-5526071592311263332->-6847263585452361362 + + + + + +-6847263585452361362->1493623177110034692 + + + + + +-6847264846649360185 + +(27, 8) + + + +-1468578046838781059->-6847264846649360185 + + + + + +-6847264846649360185->1493623177110034692 + + + + + +-6847266107846359008 + +(28, 8) + + + +-3477443014426234114->-6847266107846359008 + + + + + +-6847266107846359008->1493623177110034692 + + + + + +-6847267369043357831 + +(29, 8) + + + +6478078126746936191->-6847267369043357831 + + + + + +-6847267369043357831->1493623177110034692 + + + + + +-6847268630240356654 + +(30, 8) + + + +-7357211708903142560->-6847268630240356654 + + + + + +-6847268630240356654->1493623177110034692 + + + + + +-6847269891455996261 + +(31, 8) + + + +8400320502111359297->-6847269891455996261 + + + + + +-6847269891455996261->1493623177110034692 + + + + + +-6847250973501013916 + +(32, 8) + + + +-6730631202441397998->-6847250973501013916 + + + + + +-6847250973501013916->1493623177110034692 + + + + + +-6847252234698012739 + +(33, 8) + + + +-5772954098553506765->-6847252234698012739 + + + + + +-6847252234698012739->1493623177110034692 + + + + + +-6847253495895011562 + +(34, 8) + + + +-2202114950690208380->-6847253495895011562 + + + + + +-6847253495895011562->1493623177110034692 + + + + + +-6847254757092010385 + +(35, 8) + + + +-7357994326531260923->-6847254757092010385 + + + + + +-6847254757092010385->1493623177110034692 + + + + + +-6847256018289009208 + +(36, 8) + + + +136953021108823622->-6847256018289009208 + + + + + +-6847256018289009208->1493623177110034692 + + + + + +-6847257279486008031 + +(37, 8) + + + +1493888516157453671->-6847257279486008031 + + + + + +-6847257279486008031->1493623177110034692 + + + + + +-6847258540683006854 + +(38, 8) + + + +7155285064633981416->-6847258540683006854 + + + + + +-6847258540683006854->1493623177110034692 + + + + + +-6847259801861364893 + +(39, 8) + + + +-1276555873175892311->-6847259801861364893 + + + + + +-6847259801861364893->1493623177110034692 + + + + + +-6847240883906382548 + +(40, 8) + + + +7325330038691708186->-6847240883906382548 + + + + + +-6847240883906382548->1493623177110034692 + + + + + +-6847242145103381371 + +(41, 8) + + + +9090613057646475419->-6847242145103381371 + + + + + +-6847242145103381371->1493623177110034692 + + + + + +-6847243406300380194 + +(42, 8) + + + +-1686012982851190036->-6847243406300380194 + + + + + +-6847243406300380194->1493623177110034692 + + + + + +-6847244667497379017 + +(43, 8) + + + +-7819092055308974579->-6847244667497379017 + + + + + +-6847244667497379017->1493623177110034692 + + + + + +-6847245928694377840 + +(44, 8) + + + +-8795344833203558834->-6847245928694377840 + + + + + +-6847245928694377840->1493623177110034692 + + + + + +-6847247189891376663 + +(45, 8) + + + +-7951875832114493745->-6847247189891376663 + + + + + +-6847247189891376663->1493623177110034692 + + + + + +-6847248451088375486 + +(46, 8) + + + +6276728663850808816->-6847248451088375486 + + + + + +-6847248451088375486->1493623177110034692 + + + + + +-6847249712304015093 + +(47, 8) + + + +-4289661086082239343->-6847249712304015093 + + + + + +-6847249712304015093->1493623177110034692 + + + + + +-6847230794349032748 + +(48, 8) + + + +6254810874541667234->-6847230794349032748 + + + + + +-6847230794349032748->1493623177110034692 + + + + + +-6847291331806141301 + +(0, 9) + + + +3079740599528837393->-6847291331806141301 + + + + + +-5810866345050651649 + +simple-shuffle-combine + + + +-6847291331806141301->-5810866345050651649 + + + + + +-6847292593000810026 + +(1, 9) + + + +3313213723235198276->-6847292593000810026 + + + + + +-6847292593000810026->-5810866345050651649 + + + + + +-6847293854200138947 + +(2, 9) + + + +-3970589522996830029->-6847293854200138947 + + + + + +-6847293854200138947->-5810866345050651649 + + + + + +-6847295115394807672 + +(3, 9) + + + +4849430470623737094->-6847295115394807672 + + + + + +-6847295115394807672->-5810866345050651649 + + + + + +-6847296376594136593 + +(4, 9) + + + +6541740577366304389->-6847296376594136593 + + + + + +-6847296376594136593->-5810866345050651649 + + + + + +-6847297637788805318 + +(5, 9) + + + +-2983071401574222680->-6847297637788805318 + + + + + +-6847297637788805318->-5810866345050651649 + + + + + +-6847298898988134239 + +(6, 9) + + + +-4673260690308380185->-6847298898988134239 + + + + + +-6847298898988134239->-5810866345050651649 + + + + + +-6847300160164162180 + +(7, 9) + + + +534348247021558154->-6847300160164162180 + + + + + +-6847300160164162180->-5810866345050651649 + + + + + +-6847281242211509933 + +(8, 9) + + + +-4799897234346568263->-6847281242211509933 + + + + + +-6847281242211509933->-5810866345050651649 + + + + + +-6847282503406178658 + +(9, 9) + + + +4367645068351906220->-6847282503406178658 + + + + + +-6847282503406178658->-5810866345050651649 + + + + + +-6847283764605507579 + +(10, 9) + + + +-195749352510591717->-6847283764605507579 + + + + + +-6847283764605507579->-5810866345050651649 + + + + + +-6847285025800176304 + +(11, 9) + + + +-1192222882688343538->-6847285025800176304 + + + + + +-6847285025800176304->-5810866345050651649 + + + + + +-6847286286999505225 + +(12, 9) + + + +1014704386223945357->-6847286286999505225 + + + + + +-6847286286999505225->-5810866345050651649 + + + + + +-6847287548194173950 + +(13, 9) + + + +-4549122266439504->-6847287548194173950 + + + + + +-6847287548194173950->-5810866345050651649 + + + + + +-6847288809393502871 + +(14, 9) + + + +-1772753583852281009->-6847288809393502871 + + + + + +-6847288809393502871->-5810866345050651649 + + + + + +-6847290070606812380 + +(15, 9) + + + +-5994584163270505518->-6847290070606812380 + + + + + +-6847290070606812380->-5810866345050651649 + + + + + +-6847271152654160133 + +(16, 9) + + + +-3419855200566540127->-6847271152654160133 + + + + + +-6847271152654160133->-5810866345050651649 + + + + + +-6847272413848828858 + +(17, 9) + + + +7034060437072100500->-6847272413848828858 + + + + + +-6847272413848828858->-5810866345050651649 + + + + + +-6847273675048157779 + +(18, 9) + + + +-3403571071961885693->-6847273675048157779 + + + + + +-6847273675048157779->-5810866345050651649 + + + + + +-6847274936242826504 + +(19, 9) + + + +7035587802897479830->-6847274936242826504 + + + + + +-6847274936242826504->-5810866345050651649 + + + + + +-6847276197442155425 + +(20, 9) + + + +-1385382234702651947->-6847276197442155425 + + + + + +-6847276197442155425->-5810866345050651649 + + + + + +-6847277458636824150 + +(21, 9) + + + +-1625026760756214792->-6847277458636824150 + + + + + +-6847277458636824150->-5810866345050651649 + + + + + +-6847278719836153071 + +(22, 9) + + + +-3477286169361220297->-6847278719836153071 + + + + + +-6847278719836153071->-5810866345050651649 + + + + + +-6847279981012181012 + +(23, 9) + + + +-4674525829659598630->-6847279981012181012 + + + + + +-6847279981012181012->-5810866345050651649 + + + + + +-6847261063059528765 + +(24, 9) + + + +-3568744744482841335->-6847261063059528765 + + + + + +-6847261063059528765->-5810866345050651649 + + + + + +-6847262324254197490 + +(25, 9) + + + +-3336112252949850052->-6847262324254197490 + + + + + +-6847262324254197490->-5810866345050651649 + + + + + +-6847263585453526411 + +(26, 9) + + + +-5105389890936393557->-6847263585453526411 + + + + + +-6847263585453526411->-5810866345050651649 + + + + + +-6847264846648195136 + +(27, 9) + + + +-2957470705207124834->-6847264846648195136 + + + + + +-6847264846648195136->-5810866345050651649 + + + + + +-6847266107847524057 + +(28, 9) + + + +-1313629062832699939->-6847266107847524057 + + + + + +-6847266107847524057->-5810866345050651649 + + + + + +-6847267369042192782 + +(29, 9) + + + +5424740235828726016->-6847267369042192782 + + + + + +-6847267369042192782->-5810866345050651649 + + + + + +-6847268630241521703 + +(30, 9) + + + +-5752113260040356385->-6847268630241521703 + + + + + +-6847268630241521703->-5810866345050651649 + + + + + +-6847269891454831212 + +(31, 9) + + + +7348051249423885922->-6847269891454831212 + + + + + +-6847269891454831212->-5810866345050651649 + + + + + +-6847250973502178965 + +(32, 9) + + + +3042370101369198577->-6847250973502178965 + + + + + +-6847250973502178965->-5810866345050651649 + + + + + +-6847252234696847690 + +(33, 9) + + + +-6123809127771755740->-6847252234696847690 + + + + + +-6847252234696847690->-5810866345050651649 + + + + + +-6847253495896176611 + +(34, 9) + + + +-641899111018429805->-6847253495896176611 + + + + + +-6847253495896176611->-5810866345050651649 + + + + + +-6847254757090845336 + +(35, 9) + + + +1290651127753702->-6847254757090845336 + + + + + +-6847254757090845336->-5810866345050651649 + + + + + +-6847256018290174257 + +(36, 9) + + + +-8341642345753826203->-6847256018290174257 + + + + + +-6847256018290174257->-5810866345050651649 + + + + + +-6847257279484842982 + +(37, 9) + + + +-176746115057917304->-6847257279484842982 + + + + + +-6847257279484842982->-5810866345050651649 + + + + + +-6847258540684171903 + +(38, 9) + + + +510240498043393991->-6847258540684171903 + + + + + +-6847258540684171903->-5810866345050651649 + + + + + +-6847259801860199844 + +(39, 9) + + + +6233582989329018730->-6847259801860199844 + + + + + +-6847259801860199844->-5810866345050651649 + + + + + +-6847240883907547597 + +(40, 9) + + + +-8878932280949015655->-6847240883907547597 + + + + + +-6847240883907547597->-5810866345050651649 + + + + + +-6847242145102216322 + +(41, 9) + + + +8658849804534933644->-6847242145102216322 + + + + + +-6847242145102216322->-5810866345050651649 + + + + + +-6847243406301545243 + +(42, 9) + + + +-45018303782624261->-6847243406301545243 + + + + + +-6847243406301545243->-5810866345050651649 + + + + + +-6847244667496213968 + +(43, 9) + + + +-8872451248413940754->-6847244667496213968 + + + + + +-6847244667496213968->-5810866345050651649 + + + + + +-6847245928695542889 + +(44, 9) + + + +-7812920742476832659->-6847245928695542889 + + + + + +-6847245928695542889->-5810866345050651649 + + + + + +-6847247189890211614 + +(45, 9) + + + +7334730918875050896->-6847247189890211614 + + + + + +-6847247189890211614->-5810866345050651649 + + + + + +-6847248451089540535 + +(46, 9) + + + +-620716996724169425->-6847248451089540535 + + + + + +-6847248451089540535->-5810866345050651649 + + + + + +-6847249712302850044 + +(47, 9) + + + +-4466249485222064718->-6847249712302850044 + + + + + +-6847249712302850044->-5810866345050651649 + + + + + +-6847230794350197797 + +(48, 9) + + + +-1032448542788062591->-6847230794350197797 + + + + + +-6847230794350197797->-5810866345050651649 + + + + + +-6847291331802646154 + +(0, 10) + + + +1323560350830512868->-6847291331802646154 + + + + + +-4710198997666893974 + +simple-shuffle-combine + + + +-6847291331802646154->-4710198997666893974 + + + + + +-6847292593004305173 + +(1, 10) + + + +6275166469116070001->-6847292593004305173 + + + + + +-6847292593004305173->-4710198997666893974 + + + + + +-6847293854201303996 + +(2, 10) + + + +-2918356558383319054->-6847293854201303996 + + + + + +-6847293854201303996->-4710198997666893974 + + + + + +-6847295115393642623 + +(3, 10) + + + +-5197520953667965497->-6847295115393642623 + + + + + +-6847295115393642623->-4710198997666893974 + + + + + +-6847296376590641446 + +(4, 10) + + + +-6025279943892340152->-6847296376590641446 + + + + + +-6847296376590641446->-4710198997666893974 + + + + + +-6847297637792300465 + +(5, 10) + + + +8451186549452434661->-6847297637792300465 + + + + + +-6847297637792300465->-4710198997666893974 + + + + + +-6847298898989299288 + +(6, 10) + + + +5159508453385844006->-6847298898989299288 + + + + + +-6847298898989299288->-4710198997666893974 + + + + + +-6847300160162997131 + +(7, 10) + + + +9163767995818919595->-6847300160162997131 + + + + + +-6847300160162997131->-4710198997666893974 + + + + + +-6847281242208014786 + +(8, 10) + + + +1171952605662011212->-6847281242208014786 + + + + + +-6847281242208014786->-4710198997666893974 + + + + + +-6847282503409673805 + +(9, 10) + + + +6711742163229726745->-6847282503409673805 + + + + + +-6847282503409673805->-4710198997666893974 + + + + + +-6847283764606672628 + +(10, 10) + + + +777797939478129658->-6847283764606672628 + + + + + +-6847283764606672628->-4710198997666893974 + + + + + +-6847285025799011255 + +(11, 10) + + + +-1002129980317564113->-6847285025799011255 + + + + + +-6847285025799011255->-4710198997666893974 + + + + + +-6847286286996010078 + +(12, 10) + + + +-3029156468617589168->-6847286286996010078 + + + + + +-6847286286996010078->-4710198997666893974 + + + + + +-6847287548197669097 + +(13, 10) + + + +2830884659620163821->-6847287548197669097 + + + + + +-6847287548197669097->-4710198997666893974 + + + + + +-6847288809394667920 + +(14, 10) + + + +342152175067077166->-6847288809394667920 + + + + + +-6847288809394667920->-4710198997666893974 + + + + + +-6847290070605647331 + +(15, 10) + + + +-7544542421968240493->-6847290070605647331 + + + + + +-6847290070605647331->-4710198997666893974 + + + + + +-6847271152650664986 + +(16, 10) + + + +-4686850270006895052->-6847271152650664986 + + + + + +-6847271152650664986->-4710198997666893974 + + + + + +-6847272413852324005 + +(17, 10) + + + +1105557976937825025->-6847272413852324005 + + + + + +-6847272413852324005->-4710198997666893974 + + + + + +-6847273675049322828 + +(18, 10) + + + +-2360197071201217918->-6847273675049322828 + + + + + +-6847273675049322828->-4710198997666893974 + + + + + +-6847274936241661455 + +(19, 10) + + + +4289516849909381655->-6847274936241661455 + + + + + +-6847274936241661455->-4710198997666893974 + + + + + +-6847276197438660278 + +(20, 10) + + + +4660683066481038744->-6847276197438660278 + + + + + +-6847276197438660278->-4710198997666893974 + + + + + +-6847277458640319297 + +(21, 10) + + + +-8951522844431015883->-6847277458640319297 + + + + + +-6847277458640319297->-4710198997666893974 + + + + + +-6847278719837318120 + +(22, 10) + + + +-1314593870209461322->-6847278719837318120 + + + + + +-6847278719837318120->-4710198997666893974 + + + + + +-6847279981011015963 + +(23, 10) + + + +-5710112963197068805->-6847279981011015963 + + + + + +-6847279981011015963->-4710198997666893974 + + + + + +-6847261063056033618 + +(24, 10) + + + +-6531805814168794660->-6847261063056033618 + + + + + +-6847261063056033618->-4710198997666893974 + + + + + +-6847262324257692637 + +(25, 10) + + + +-1580236089230591127->-6847262324257692637 + + + + + +-6847262324257692637->-4710198997666893974 + + + + + +-6847263585454691460 + +(26, 10) + + + +-2932739384969944182->-6847263585454691460 + + + + + +-6847263585454691460->-4710198997666893974 + + + + + +-6847264846647030087 + +(27, 10) + + + +-5614428255585862209->-6847264846647030087 + + + + + +-6847264846647030087->-4710198997666893974 + + + + + +-6847266107844028910 + +(28, 10) + + + +-6461168746197626464->-6847266107844028910 + + + + + +-6847266107844028910->-4710198997666893974 + + + + + +-6847267369045687929 + +(29, 10) + + + +-7748121100574324675->-6847267369045687929 + + + + + +-6847267369045687929->-4710198997666893974 + + + + + +-6847268630242686752 + +(30, 10) + + + +4068536220299008190->-6847268630242686752 + + + + + +-6847268630242686752->-4710198997666893974 + + + + + +-6847269891453666163 + +(31, 10) + + + +6922846524790936547->-6847269891453666163 + + + + + +-6847269891453666163->-4710198997666893974 + + + + + +-6847250973498683818 + +(32, 10) + + + +-8184394169512837948->-6847250973498683818 + + + + + +-6847250973498683818->-4710198997666893974 + + + + + +-6847252234700342837 + +(33, 10) + + + +-316626859096925615->-6847252234700342837 + + + + + +-6847252234700342837->-4710198997666893974 + + + + + +-6847253495897341660 + +(34, 10) + + + +-8152037945810700846->-6847253495897341660 + + + + + +-6847253495897341660->-4710198997666893974 + + + + + +-6847254757089680287 + +(35, 10) + + + +5746656406321535143->-6847254757089680287 + + + + + +-6847254757089680287->-4710198997666893974 + + + + + +-6847256018286679110 + +(36, 10) + + + +4929165357978600488->-6847256018286679110 + + + + + +-6847256018286679110->-4710198997666893974 + + + + + +-6847257279488338129 + +(37, 10) + + + +-5678800229175163195->-6847257279488338129 + + + + + +-6847257279488338129->-4710198997666893974 + + + + + +-6847258540685336952 + +(38, 10) + + + +-8200114494783027450->-6847258540685336952 + + + + + +-6847258540685336952->-4710198997666893974 + + + + + +-6847259801859034795 + +(39, 10) + + + +-4255816353658239861->-6847259801859034795 + + + + + +-6847259801859034795->-4710198997666893974 + + + + + +-6847240883904052450 + +(40, 10) + + + +5534360125878440236->-6847240883904052450 + + + + + +-6847240883904052450->-4710198997666893974 + + + + + +-6847242145105711469 + +(41, 10) + + + +-7511980271019952647->-6847242145105711469 + + + + + +-6847242145105711469->-4710198997666893974 + + + + + +-6847243406302710292 + +(42, 10) + + + +179347482624221914->-6847243406302710292 + + + + + +-6847243406302710292->-4710198997666893974 + + + + + +-6847244667495048919 + +(43, 10) + + + +6402985717915427087->-6847244667495048919 + + + + + +-6847244667495048919->-4710198997666893974 + + + + + +-6847245928692047742 + +(44, 10) + + + +4309319220630314032->-6847245928692047742 + + + + + +-6847245928692047742->-4710198997666893974 + + + + + +-6847247189893706761 + +(45, 10) + + + +-5851682614305490995->-6847247189893706761 + + + + + +-6847247189893706761->-4710198997666893974 + + + + + +-6847248451090705584 + +(46, 10) + + + +-9098224101585368050->-6847248451090705584 + + + + + +-6847248451090705584->-4710198997666893974 + + + + + +-6847249712301684995 + +(47, 10) + + + +-6156300142636951693->-6847249712301684995 + + + + + +-6847249712301684995->-4710198997666893974 + + + + + +-6847230794346702650 + +(48, 10) + + + +-3901867643125953516->-6847230794346702650 + + + + + +-6847230794346702650->-4710198997666893974 + + + + + +-6847291331803811203 + +(0, 11) + + + +1546817770183238643->-6847291331803811203 + + + + + +-3687284701743748299 + +simple-shuffle-combine + + + +-6847291331803811203->-3687284701743748299 + + + + + +-6847292593003140124 + +(1, 11) + + + +5231792478366593426->-6847292593003140124 + + + + + +-6847292593003140124->-3687284701743748299 + + + + + +-6847293854202469045 + +(2, 11) + + + +-187003906035455279->-6847293854202469045 + + + + + +-6847293854202469045->-3687284701743748299 + + + + + +-6847295115392477574 + +(3, 11) + + + +2303458327306631144->-6847295115392477574 + + + + + +-6847295115392477574->-3687284701743748299 + + + + + +-6847296376591806495 + +(4, 11) + + + +3952815953405958439->-6847296376591806495 + + + + + +-6847296376591806495->-3687284701743748299 + + + + + +-6847297637791135416 + +(5, 11) + + + +6288482063355468486->-6847297637791135416 + + + + + +-6847297637791135416->-3687284701743748299 + + + + + +-6847298898990464337 + +(6, 11) + + + +7393149103809414981->-6847298898990464337 + + + + + +-6847298898990464337->-3687284701743748299 + + + + + +-6847300160161832082 + +(7, 11) + + + +-2556760286676292196->-6847300160161832082 + + + + + +-6847300160161832082->-3687284701743748299 + + + + + +-6847281242209179835 + +(8, 11) + + + +3406683789397687387->-6847281242209179835 + + + + + +-6847281242209179835->-3687284701743748299 + + + + + +-6847282503408508756 + +(9, 11) + + + +5747053807415849370->-6847282503408508756 + + + + + +-6847282503408508756->-3687284701743748299 + + + + + +-6847283764607837677 + +(10, 11) + + + +1812314092581811833->-6847283764607837677 + + + + + +-6847283764607837677->-3687284701743748299 + + + + + +-6847285025797846206 + +(11, 11) + + + +-2045508512579843088->-6847285025797846206 + + + + + +-6847285025797846206->-3687284701743748299 + + + + + +-6847286286997175127 + +(12, 11) + + + +-2056712994393326193->-6847286286997175127 + + + + + +-6847286286997175127->-3687284701743748299 + + + + + +-6847287548196504048 + +(13, 11) + + + +2606502101454308046->-6847287548196504048 + + + + + +-6847287548196504048->-3687284701743748299 + + + + + +-6847288809395832969 + +(14, 11) + + + +-8287282549571658675->-6847288809395832969 + + + + + +-6847288809395832969->-3687284701743748299 + + + + + +-6847290070604482282 + +(15, 11) + + + +-7838771230630022268->-6847290070604482282 + + + + + +-6847290070604482282->-3687284701743748299 + + + + + +-6847271152651830035 + +(16, 11) + + + +-3687834559722457277->-6847271152651830035 + + + + + +-6847271152651830035->-3687284701743748299 + + + + + +-6847272413851158956 + +(17, 11) + + + +52184934233174050->-6847272413851158956 + + + + + +-6847272413851158956->-3687284701743748299 + + + + + +-6847273675050487877 + +(18, 11) + + + +746686772352533857->-6847273675050487877 + + + + + +-6847273675050487877->-3687284701743748299 + + + + + +-6847274936240496406 + +(19, 11) + + + +-5599888682697819336->-6847274936240496406 + + + + + +-6847274936240496406->-3687284701743748299 + + + + + +-6847276197439825327 + +(20, 11) + + + +5695202625987788919->-6847276197439825327 + + + + + +-6847276197439825327->-3687284701743748299 + + + + + +-6847277458639154248 + +(21, 11) + + + +-258907002931702442->-6847277458639154248 + + + + + +-6847277458639154248->-3687284701743748299 + + + + + +-6847278719838483169 + +(22, 11) + + + +1882525018418876053->-6847278719838483169 + + + + + +-6847278719838483169->-3687284701743748299 + + + + + +-6847279981009850914 + +(23, 11) + + + +1801129726996889836->-6847279981009850914 + + + + + +-6847279981009850914->-3687284701743748299 + + + + + +-6847261063057198667 + +(24, 11) + + + +-5487322364521822485->-6847261063057198667 + + + + + +-6847261063057198667->-3687284701743748299 + + + + + +-6847262324256527588 + +(25, 11) + + + +-2993746873863148502->-6847262324256527588 + + + + + +-6847262324256527588->-3687284701743748299 + + + + + +-6847263585455856509 + +(26, 11) + + + +9145617873162001865->-6847263585455856509 + + + + + +-6847263585455856509->-3687284701743748299 + + + + + +-6847264846645865038 + +(27, 11) + + + +-5550802954723617984->-6847264846645865038 + + + + + +-6847264846645865038->-3687284701743748299 + + + + + +-6847266107845193959 + +(28, 11) + + + +-6669011349474027489->-6847266107845193959 + + + + + +-6847266107845193959->-3687284701743748299 + + + + + +-6847267369044522880 + +(29, 11) + + + +8463873892559318366->-6847267369044522880 + + + + + +-6847267369044522880->-3687284701743748299 + + + + + +-6847268630243851801 + +(30, 11) + + + +-3837943264414376035->-6847268630243851801 + + + + + +-6847268630243851801->-3687284701743748299 + + + + + +-6847269891452501114 + +(31, 11) + + + +4807936225449961172->-6847269891452501114 + + + + + +-6847269891452501114->-3687284701743748299 + + + + + +-6847250973499848867 + +(32, 11) + + + +-7065521225826569773->-6847250973499848867 + + + + + +-6847250973499848867->-3687284701743748299 + + + + + +-6847252234699177788 + +(33, 11) + + + +-3534902409588968590->-6847252234699177788 + + + + + +-6847252234699177788->-3687284701743748299 + + + + + +-6847253495898506709 + +(34, 11) + + + +3529568452602452145->-6847253495898506709 + + + + + +-6847253495898506709->-3687284701743748299 + + + + + +-6847254757088515238 + +(35, 11) + + + +5964498079323840968->-6847254757088515238 + + + + + +-6847254757088515238->-3687284701743748299 + + + + + +-6847256018287844159 + +(36, 11) + + + +5972540483837811463->-6847256018287844159 + + + + + +-6847256018287844159->-3687284701743748299 + + + + + +-6847257279487173080 + +(37, 11) + + + +-7219921987920679770->-6847257279487173080 + + + + + +-6847257279487173080->-3687284701743748299 + + + + + +-6847258540686502001 + +(38, 11) + + + +3471799864563714341->-6847258540686502001 + + + + + +-6847258540686502001->-3687284701743748299 + + + + + +-6847259801857869746 + +(39, 11) + + + +-5303754197183279236->-6847259801857869746 + + + + + +-6847259801857869746->-3687284701743748299 + + + + + +-6847240883905217499 + +(40, 11) + + + +5417549409115684411->-6847240883905217499 + + + + + +-6847240883905217499->-3687284701743748299 + + + + + +-6847242145104546420 + +(41, 11) + + + +-9060830207090393222->-6847242145104546420 + + + + + +-6847242145104546420->-3687284701743748299 + + + + + +-6847243406303875341 + +(42, 11) + + + +2934278498063494489->-6847243406303875341 + + + + + +-6847243406303875341->-3687284701743748299 + + + + + +-6847244667493883870 + +(43, 11) + + + +5367361206712271312->-6847244667493883870 + + + + + +-6847244667493883870->-3687284701743748299 + + + + + +-6847245928693212791 + +(44, 11) + + + +5274012152286026607->-6847245928693212791 + + + + + +-6847245928693212791->-3687284701743748299 + + + + + +-6847247189892541712 + +(45, 11) + + + +-6979468681367090770->-6847247189892541712 + + + + + +-6847247189892541712->-3687284701743748299 + + + + + +-6847248451091870633 + +(46, 11) + + + +-8739601758022238675->-6847248451091870633 + + + + + +-6847248451091870633->-3687284701743748299 + + + + + +-6847249712300519946 + +(47, 11) + + + +-7128702755145107868->-6847249712300519946 + + + + + +-6847249712300519946->-3687284701743748299 + + + + + +-6847230794347867699 + +(48, 11) + + + +-3499332406775237341->-6847230794347867699 + + + + + +-6847230794347867699->-3687284701743748299 + + + + + +-6847291331800316056 + +(0, 12) + + + +-1217582318854381082->-6847291331800316056 + + + + + +6338118214775568592 + +simple-shuffle-combine + + + +-6847291331800316056->6338118214775568592 + + + + + +-6847292592997314879 + +(1, 12) + + + +859273565801279751->-6847292592997314879 + + + + + +-6847292592997314879->6338118214775568592 + + + + + +-6847293854194313702 + +(2, 12) + + + +-3302229621990768504->-6847293854194313702 + + + + + +-6847293854194313702->6338118214775568592 + + + + + +-6847295115400632917 + +(3, 12) + + + +-6485099818142743247->-6847295115400632917 + + + + + +-6847295115400632917->6338118214775568592 + + + + + +-6847296376597631740 + +(4, 12) + + + +8250103647880590514->-6847296376597631740 + + + + + +-6847296376597631740->6338118214775568592 + + + + + +-6847297637794630563 + +(5, 12) + + + +-6943402656622703405->-6847297637794630563 + + + + + +-6847297637794630563->6338118214775568592 + + + + + +-6847298898991629386 + +(6, 12) + + + +-7909711097203421660->-6847298898991629386 + + + + + +-6847298898991629386->6338118214775568592 + + + + + +-6847300160160667033 + +(7, 12) + + + +-2410994982461369571->-6847300160160667033 + + + + + +-6847300160160667033->6338118214775568592 + + + + + +-6847281242205684688 + +(8, 12) + + + +-1807360966765329938->-6847281242205684688 + + + + + +-6847281242205684688->6338118214775568592 + + + + + +-6847282503402683511 + +(9, 12) + + + +1524425491201661295->-6847282503402683511 + + + + + +-6847282503402683511->6338118214775568592 + + + + + +-6847283764599682334 + +(10, 12) + + + +-6325228452822740592->-6847283764599682334 + + + + + +-6847283764599682334->6338118214775568592 + + + + + +-6847285025806001549 + +(11, 12) + + + +8803048626519589337->-6847285025806001549 + + + + + +-6847285025806001549->6338118214775568592 + + + + + +-6847286287003000372 + +(12, 12) + + + +4586014001545610682->-6847286287003000372 + + + + + +-6847286287003000372->6338118214775568592 + + + + + +-6847287548199999195 + +(13, 12) + + + +4293542852048915771->-6847287548199999195 + + + + + +-6847287548199999195->6338118214775568592 + + + + + +-6847288809396998018 + +(14, 12) + + + +2256285881579871116->-6847288809396998018 + + + + + +-6847288809396998018->6338118214775568592 + + + + + +-6847290070603317233 + +(15, 12) + + + +7830765746973559973->-6847290070603317233 + + + + + +-6847290070603317233->6338118214775568592 + + + + + +-6847271152648334888 + +(16, 12) + + + +-8911552886174725002->-6847271152648334888 + + + + + +-6847271152648334888->6338118214775568592 + + + + + +-6847272413845333711 + +(17, 12) + + + +-4240406132102720425->-6847272413845333711 + + + + + +-6847272413845333711->6338118214775568592 + + + + + +-6847273675042332534 + +(18, 12) + + + +-436632482171542568->-6847273675042332534 + + + + + +-6847273675042332534->6338118214775568592 + + + + + +-6847274936248651749 + +(19, 12) + + + +3965047606275786689->-6847274936248651749 + + + + + +-6847274936248651749->6338118214775568592 + + + + + +-6847276197445650572 + +(20, 12) + + + +-6625650574147877822->-6847276197445650572 + + + + + +-6847276197445650572->6338118214775568592 + + + + + +-6847277458642649395 + +(21, 12) + + + +2584313638331957283->-6847277458642649395 + + + + + +-6847277458642649395->6338118214775568592 + + + + + +-6847278719839648218 + +(22, 12) + + + +1664683301076346228->-6847278719839648218 + + + + + +-6847278719839648218->6338118214775568592 + + + + + +-6847279981008685865 + +(23, 12) + + + +8511945258326802861->-6847279981008685865 + + + + + +-6847279981008685865->6338118214775568592 + + + + + +-6847261063053703520 + +(24, 12) + + + +6678305913371745406->-6847261063053703520 + + + + + +-6847261063053703520->6338118214775568592 + + + + + +-6847262324250702343 + +(25, 12) + + + +-8555669404345336577->-6847262324250702343 + + + + + +-6847262324250702343->6338118214775568592 + + + + + +-6847263585447701166 + +(26, 12) + + + +7360920541637365984->-6847263585447701166 + + + + + +-6847263585447701166->6338118214775568592 + + + + + +-6847264846654020381 + +(27, 12) + + + +5241114389059769641->-6847264846654020381 + + + + + +-6847264846654020381->6338118214775568592 + + + + + +-6847266107851019204 + +(28, 12) + + + +2077880023070093386->-6847266107851019204 + + + + + +-6847266107851019204->6338118214775568592 + + + + + +-6847267369048018027 + +(29, 12) + + + +-7458972949195241525->-6847267369048018027 + + + + + +-6847267369048018027->6338118214775568592 + + + + + +-6847268630245016850 + +(30, 12) + + + +-2913322583994239460->-6847268630245016850 + + + + + +-6847268630245016850->6338118214775568592 + + + + + +-6847269891451336065 + +(31, 12) + + + +1619425237804542197->-6847269891451336065 + + + + + +-6847269891451336065->6338118214775568592 + + + + + +-6847250973496353720 + +(32, 12) + + + +6112037874985419718->-6847250973496353720 + + + + + +-6847250973496353720->6338118214775568592 + + + + + +-6847252234693352543 + +(33, 12) + + + +8198857650907128551->-6847252234693352543 + + + + + +-6847252234693352543->6338118214775568592 + + + + + +-6847253495890351366 + +(34, 12) + + + +-7319257512095280280->-6847253495890351366 + + + + + +-6847253495890351366->6338118214775568592 + + + + + +-6847254757096670581 + +(35, 12) + + + +-2768528627869738223->-6847254757096670581 + + + + + +-6847254757096670581->6338118214775568592 + + + + + +-6847256018293669404 + +(36, 12) + + + +4699563700969823122->-6847256018293669404 + + + + + +-6847256018293669404->6338118214775568592 + + + + + +-6847257279490668227 + +(37, 12) + + + +5276162369028680371->-6847257279490668227 + + + + + +-6847257279490668227->6338118214775568592 + + + + + +-6847258540687667050 + +(38, 12) + + + +4448668800233869316->-6847258540687667050 + + + + + +-6847258540687667050->6338118214775568592 + + + + + +-6847259801856704697 + +(39, 12) + + + +-8476072205171518211->-6847259801856704697 + + + + + +-6847259801856704697->6338118214775568592 + + + + + +-6847240883901722352 + +(40, 12) + + + +174585261609844686->-6847240883901722352 + + + + + +-6847240883901722352->6338118214775568592 + + + + + +-6847242145098721175 + +(41, 12) + + + +3535022293046256719->-6847242145098721175 + + + + + +-6847242145098721175->6338118214775568592 + + + + + +-6847243406295719998 + +(42, 12) + + + +-6722534813472699536->-6847243406295719998 + + + + + +-6847243406295719998->6338118214775568592 + + + + + +-6847244667502039213 + +(43, 12) + + + +-2299645038939435079->-6847244667502039213 + + + + + +-6847244667502039213->6338118214775568592 + + + + + +-6847245928699038036 + +(44, 12) + + + +-4281530416880786534->-6847245928699038036 + + + + + +-6847245928699038036->6338118214775568592 + + + + + +-6847247189896036859 + +(45, 12) + + + +4250938072198364955->-6847247189896036859 + + + + + +-6847247189896036859->6338118214775568592 + + + + + +-6847248451093035682 + +(46, 12) + + + +-7695086518668990100->-6847248451093035682 + + + + + +-6847248451093035682->6338118214775568592 + + + + + +-6847249712299354897 + +(47, 12) + + + +-3289950943374392443->-6847249712299354897 + + + + + +-6847249712299354897->6338118214775568592 + + + + + +-6847230794344372552 + +(48, 12) + + + +1775453080013454934->-6847230794344372552 + + + + + +-6847230794344372552->6338118214775568592 + + + + + +-6847291331801481105 + +(0, 13) + + + +-169756980801600507->-6847291331801481105 + + + + + +-983835332281493349 + +simple-shuffle-combine + + + +-6847291331801481105->-983835332281493349 + + + + + +-6847292592996149830 + +(1, 13) + + + +-2320928794648856024->-6847292592996149830 + + + + + +-6847292592996149830->-983835332281493349 + + + + + +-6847293854195478751 + +(2, 13) + + + +36405955408999271->-6847293854195478751 + + + + + +-6847293854195478751->-983835332281493349 + + + + + +-6847295115399467868 + +(3, 13) + + + +-8077889546049621422->-6847295115399467868 + + + + + +-6847295115399467868->-983835332281493349 + + + + + +-6847296376598796789 + +(4, 13) + + + +-7540791197038294127->-6847296376598796789 + + + + + +-6847296376598796789->-983835332281493349 + + + + + +-6847297637793465514 + +(5, 13) + + + +1535212815292615620->-6847297637793465514 + + + + + +-6847297637793465514->-983835332281493349 + + + + + +-6847298898992794435 + +(6, 13) + + + +-8127520997127035085->-6847298898992794435 + + + + + +-6847298898992794435->-983835332281493349 + + + + + +-6847300160159501984 + +(7, 13) + + + +-3317887246322628546->-6847300160159501984 + + + + + +-6847300160159501984->-983835332281493349 + + + + + +-6847281242206849737 + +(8, 13) + + + +-1953109529185534963->-6847281242206849737 + + + + + +-6847281242206849737->-983835332281493349 + + + + + +-6847282503401518462 + +(9, 13) + + + +-1663785940783932880->-6847282503401518462 + + + + + +-6847282503401518462->-983835332281493349 + + + + + +-6847283764600847383 + +(10, 13) + + + +-3593880342056960817->-6847283764600847383 + + + + + +-6847283764600847383->-983835332281493349 + + + + + +-6847285025804836500 + +(11, 13) + + + +5566206497322902362->-6847285025804836500 + + + + + +-6847285025804836500->-983835332281493349 + + + + + +-6847286287004165421 + +(12, 13) + + + +6684329456651443257->-6847286287004165421 + + + + + +-6847286287004165421->-983835332281493349 + + + + + +-6847287548198834146 + +(13, 13) + + + +3236746071205885996->-6847287548198834146 + + + + + +-6847287548198834146->-983835332281493349 + + + + + +-6847288809398163067 + +(14, 13) + + + +3309927778721288091->-6847288809398163067 + + + + + +-6847288809398163067->-983835332281493349 + + + + + +-6847290070602152184 + +(15, 13) + + + +6853880041761406598->-6847290070602152184 + + + + + +-6847290070602152184->-983835332281493349 + + + + + +-6847271152649499937 + +(16, 13) + + + +-7859302051307795627->-6847271152649499937 + + + + + +-6847271152649499937->-983835332281493349 + + + + + +-6847272413844168662 + +(17, 13) + + + +2633517564721947000->-6847272413844168662 + + + + + +-6847272413844168662->-983835332281493349 + + + + + +-6847273675043497583 + +(18, 13) + + + +-8922999661677477193->-6847273675043497583 + + + + + +-6847273675043497583->-983835332281493349 + + + + + +-6847274936247486700 + +(19, 13) + + + +-7668511170842841886->-6847274936247486700 + + + + + +-6847274936247486700->-983835332281493349 + + + + + +-6847276197446815621 + +(20, 13) + + + +4314275136795057953->-6847276197446815621 + + + + + +-6847276197446815621->-983835332281493349 + + + + + +-6847277458641484346 + +(21, 13) + + + +-7265096771664208108->-6847277458641484346 + + + + + +-6847277458641484346->-983835332281493349 + + + + + +-6847278719840813267 + +(22, 13) + + + +4497030593911798403->-6847278719840813267 + + + + + +-6847278719840813267->-983835332281493349 + + + + + +-6847279981007520816 + +(23, 13) + + + +8729733880398822286->-6847279981007520816 + + + + + +-6847279981007520816->-983835332281493349 + + + + + +-6847261063054868569 + +(24, 13) + + + +7604057291639353181->-6847261063054868569 + + + + + +-6847261063054868569->-983835332281493349 + + + + + +-6847262324249537294 + +(25, 13) + + + +7135036429757416064->-6847262324249537294 + + + + + +-6847262324249537294->-983835332281493349 + + + + + +-6847263585448866215 + +(26, 13) + + + +8835383288251652959->-6847263585448866215 + + + + + +-6847263585448866215->-983835332281493349 + + + + + +-6847264846652855332 + +(27, 13) + + + +2585265160303743466->-6847264846652855332 + + + + + +-6847264846652855332->-983835332281493349 + + + + + +-6847266107852184253 + +(28, 13) + + + +3122362380821729161->-6847266107852184253 + + + + + +-6847266107852184253->-983835332281493349 + + + + + +-6847267369046852978 + +(29, 13) + + + +-8432520205694463300->-6847267369046852978 + + + + + +-6847267369046852978->-983835332281493349 + + + + + +-6847268630246181899 + +(30, 13) + + + +-1868840225116777685->-6847268630246181899 + + + + + +-6847268630246181899->-983835332281493349 + + + + + +-6847269891450171016 + +(31, 13) + + + +1828354924844864022->-6847269891450171016 + + + + + +-6847269891450171016->-983835332281493349 + + + + + +-6847250973497518769 + +(32, 13) + + + +-9165691997419076123->-6847250973497518769 + + + + + +-6847250973497518769->-983835332281493349 + + + + + +-6847252234692187494 + +(33, 13) + + + +6657758362747911176->-6847252234692187494 + + + + + +-6847252234692187494->-983835332281493349 + + + + + +-6847253495891516415 + +(34, 13) + + + +-7465860114133368505->-6847253495891516415 + + + + + +-6847253495891516415->-983835332281493349 + + + + + +-6847254757095505532 + +(35, 13) + + + +5057363460798001202->-6847254757095505532 + + + + + +-6847254757095505532->-983835332281493349 + + + + + +-6847256018294834453 + +(36, 13) + + + +-3897769963834748303->-6847256018294834453 + + + + + +-6847256018294834453->-983835332281493349 + + + + + +-6847257279489503178 + +(37, 13) + + + +4293756102050098596->-6847257279489503178 + + + + + +-6847257279489503178->-983835332281493349 + + + + + +-6847258540688832099 + +(38, 13) + + + +-1223838892427852525->-6847258540688832099 + + + + + +-6847258540688832099->-983835332281493349 + + + + + +-6847259801855539648 + +(39, 13) + + + +8896140484583692830->-6847259801855539648 + + + + + +-6847259801855539648->-983835332281493349 + + + + + +-6847240883902887401 + +(40, 13) + + + +4489876640776609261->-6847240883902887401 + + + + + +-6847240883902887401->-983835332281493349 + + + + + +-6847242145097556126 + +(41, 13) + + + +1974774643715100944->-6847242145097556126 + + + + + +-6847242145097556126->-983835332281493349 + + + + + +-6847243406296885047 + +(42, 13) + + + +-5745666950870606161->-6847243406296885047 + + + + + +-6847243406296885047->-983835332281493349 + + + + + +-6847244667500874164 + +(43, 13) + + + +-3924030173880096454->-6847244667500874164 + + + + + +-6847244667500874164->-983835332281493349 + + + + + +-6847245928700203085 + +(44, 13) + + + +-4418401631956637159->-6847245928700203085 + + + + + +-6847245928700203085->-983835332281493349 + + + + + +-6847247189894871810 + +(45, 13) + + + +-6637080864966578420->-6847247189894871810 + + + + + +-6847247189894871810->-983835332281493349 + + + + + +-6847248451094200731 + +(46, 13) + + + +-3287667131647784325->-6847248451094200731 + + + + + +-6847248451094200731->-983835332281493349 + + + + + +-6847249712298189848 + +(47, 13) + + + +-4262407703142193818->-6847249712298189848 + + + + + +-6847249712298189848->-983835332281493349 + + + + + +-6847230794345537601 + +(48, 13) + + + +-4878700316252415691->-6847230794345537601 + + + + + +-6847230794345537601->-983835332281493349 + + + + + +-6847291331797985958 + +(0, 14) + + + +-6577335906397208632->-6847291331797985958 + + + + + +38829656575316326 + +simple-shuffle-combine + + + +-6847291331797985958->38829656575316326 + + + + + +-6847292592999644977 + +(1, 14) + + + +1197194994763788901->-6847292592999644977 + + + + + +-6847292592999644977->38829656575316326 + + + + + +-6847293854196643800 + +(2, 14) + + + +-282609514560785754->-6847293854196643800 + + + + + +-6847293854196643800->38829656575316326 + + + + + +-6847295115398302819 + +(3, 14) + + + +-7868938592294722797->-6847295115398302819 + + + + + +-6847295115398302819->38829656575316326 + + + + + +-6847296376595301642 + +(4, 14) + + + +-1510450120903569052->-6847296376595301642 + + + + + +-6847296376595301642->38829656575316326 + + + + + +-6847297637796960661 + +(5, 14) + + + +5557883608910114545->-6847297637796960661 + + + + + +-6847297637796960661->38829656575316326 + + + + + +-6847298898993959484 + +(6, 14) + + + +-5316378847773530510->-6847298898993959484 + + + + + +-6847298898993959484->38829656575316326 + + + + + +-6847300160158336935 + +(7, 14) + + + +3498586838952249695->-6847300160158336935 + + + + + +-6847300160158336935->38829656575316326 + + + + + +-6847281242203354590 + +(8, 14) + + + +-5978792128170813488->-6847281242203354590 + + + + + +-6847281242203354590->38829656575316326 + + + + + +-6847282503405013609 + +(9, 14) + + + +-5457396988436939155->-6847282503405013609 + + + + + +-6847282503405013609->38829656575316326 + + + + + +-6847283764602012432 + +(10, 14) + + + +-3785077104280770642->-6847283764602012432 + + + + + +-6847283764602012432->38829656575316326 + + + + + +-6847285025803671451 + +(11, 14) + + + +4521722977822756987->-6847285025803671451 + + + + + +-6847285025803671451->38829656575316326 + + + + + +-6847286287000670274 + +(12, 14) + + + +2564698968433181132->-6847286287000670274 + + + + + +-6847286287000670274->38829656575316326 + + + + + +-6847287548202329293 + +(13, 14) + + + +8426119171842989721->-6847287548202329293 + + + + + +-6847287548202329293->38829656575316326 + + + + + +-6847288809399328116 + +(14, 14) + + + +7616343511995871866->-6847288809399328116 + + + + + +-6847288809399328116->38829656575316326 + + + + + +-6847290070600987135 + +(15, 14) + + + +3684918935761502023->-6847290070600987135 + + + + + +-6847290070600987135->38829656575316326 + + + + + +-6847271152646004790 + +(16, 14) + + + +6560360780886250264->-6847271152646004790 + + + + + +-6847271152646004790->38829656575316326 + + + + + +-6847272413847663809 + +(17, 14) + + + +-2782160070110272075->-6847272413847663809 + + + + + +-6847272413847663809->38829656575316326 + + + + + +-6847273675044662632 + +(18, 14) + + + +-9140845919424303818->-6847273675044662632 + + + + + +-6847273675044662632->38829656575316326 + + + + + +-6847274936246321651 + +(19, 14) + + + +-8567667911312565661->-6847274936246321651 + + + + + +-6847274936246321651->38829656575316326 + + + + + +-6847276197443320474 + +(20, 14) + + + +-904996418368944972->-6847276197443320474 + + + + + +-6847276197443320474->38829656575316326 + + + + + +-6847277458644979493 + +(21, 14) + + + +7939641765641320833->-6847277458644979493 + + + + + +-6847277458644979493->38829656575316326 + + + + + +-6847278719841978316 + +(22, 14) + + + +-2938815174565533438->-6847278719841978316 + + + + + +-6847278719841978316->38829656575316326 + + + + + +-6847279981006355767 + +(23, 14) + + + +7106917565802740911->-6847279981006355767 + + + + + +-6847279981006355767->38829656575316326 + + + + + +-6847261063051373422 + +(24, 14) + + + +4084973661638241056->-6847261063051373422 + + + + + +-6847261063051373422->38829656575316326 + + + + + +-6847262324253032441 + +(25, 14) + + + +-4261845769901365827->-6847262324253032441 + + + + + +-6847262324253032441->38829656575316326 + + + + + +-6847263585450031264 + +(26, 14) + + + +-8492486739899743682->-6847263585450031264 + + + + + +-6847263585450031264->38829656575316326 + + + + + +-6847264846651690283 + +(27, 14) + + + +1090819086193396491->-6847264846651690283 + + + + + +-6847264846651690283->38829656575316326 + + + + + +-6847266107848689106 + +(28, 14) + + + +-937310084744032164->-6847266107848689106 + + + + + +-6847266107848689106->38829656575316326 + + + + + +-6847267369050348125 + +(29, 14) + + + +-3234287094775229975->-6847267369050348125 + + + + + +-6847267369050348125->38829656575316326 + + + + + +-6847268630247346948 + +(30, 14) + + + +2446470734623623690->-6847268630247346948 + + + + + +-6847268630247346948->38829656575316326 + + + + + +-6847269891449005967 + +(31, 14) + + + +8548062354837501847->-6847269891449005967 + + + + + +-6847269891449005967->38829656575316326 + + + + + +-6847250973494023622 + +(32, 14) + + + +-6523443078926671448->-6847250973494023622 + + + + + +-6847250973494023622->38829656575316326 + + + + + +-6847252234695682641 + +(33, 14) + + + +2345915623735689285->-6847252234695682641 + + + + + +-6847252234695682641->38829656575316326 + + + + + +-6847253495892681464 + +(34, 14) + + + +-4669016640590577530->-6847253495892681464 + + + + + +-6847253495892681464->38829656575316326 + + + + + +-6847254757094340483 + +(35, 14) + + + +-5950480900074754573->-6847254757094340483 + + + + + +-6847254757094340483->38829656575316326 + + + + + +-6847256018291339306 + +(36, 14) + + + +-9117036978490532028->-6847256018291339306 + + + + + +-6847256018291339306->38829656575316326 + + + + + +-6847257279492998325 + +(37, 14) + + + +-7718683402909102895->-6847257279492998325 + + + + + +-6847257279492998325->38829656575316326 + + + + + +-6847258540689997148 + +(38, 14) + + + +8673353518418320466->-6847258540689997148 + + + + + +-6847258540689997148->38829656575316326 + + + + + +-6847259801854374599 + +(39, 14) + + + +-8690851117108308161->-6847259801854374599 + + + + + +-6847259801854374599->38829656575316326 + + + + + +-6847240883899392254 + +(40, 14) + + + +-1224069255223990864->-6847240883899392254 + + + + + +-6847240883899392254->38829656575316326 + + + + + +-6847242145101051273 + +(41, 14) + + + +7685302546944526669->-6847242145101051273 + + + + + +-6847242145101051273->38829656575316326 + + + + + +-6847243406298050096 + +(42, 14) + + + +-4260116181929974386->-6847243406298050096 + + + + + +-6847243406298050096->38829656575316326 + + + + + +-6847244667499709115 + +(43, 14) + + + +-4967372382665515429->-6847244667499709115 + + + + + +-6847244667499709115->38829656575316326 + + + + + +-6847245928696707938 + +(44, 14) + + + +-7317852986772864084->-6847245928696707938 + + + + + +-6847245928696707938->38829656575316326 + + + + + +-6847247189898366957 + +(45, 14) + + + +-2682149211510153095->-6847247189898366957 + + + + + +-6847247189898366957->38829656575316326 + + + + + +-6847248451095365780 + +(46, 14) + + + +-2314119811946422950->-6847248451095365780 + + + + + +-6847248451095365780->38829656575316326 + + + + + +-6847249712297024799 + +(47, 14) + + + +-5298031160156523993->-6847249712297024799 + + + + + +-6847249712297024799->38829656575316326 + + + + + +-6847230794342042454 + +(48, 14) + + + +7720910888120907000->-6847230794342042454 + + + + + +-6847230794342042454->38829656575316326 + + + + + +-6847291331799151007 + +(0, 15) + + + +-5525106383658944857->-6847291331799151007 + + + + + +1139456137791624401 + +simple-shuffle-combine + + + +-6847291331799151007->1139456137791624401 + + + + + +-6847292592998479928 + +(1, 15) + + + +1903755951265555526->-6847292592998479928 + + + + + +-6847292592998479928->1139456137791624401 + + + + + +-6847293854197808849 + +(2, 15) + + + +213584497347278021->-6847293854197808849 + + + + + +-6847293854197808849->1139456137791624401 + + + + + +-6847295115397137770 + +(3, 15) + + + +-1149231197722302972->-6847295115397137770 + + + + + +-6847295115397137770->1139456137791624401 + + + + + +-6847296376596466691 + +(4, 15) + + + +8387850645911668339->-6847296376596466691 + + + + + +-6847296376596466691->1139456137791624401 + + + + + +-6847297637795795612 + +(5, 15) + + + +-4258340346833179630->-6847297637795795612 + + + + + +-6847297637795795612->1139456137791624401 + + + + + +-6847298898995124533 + +(6, 15) + + + +5619158861487894865->-6847298898995124533 + + + + + +-6847298898995124533->1139456137791624401 + + + + + +-6847300160157171886 + +(7, 15) + + + +-6477424979172241696->-6847300160157171886 + + + + + +-6847300160157171886->1139456137791624401 + + + + + +-6847281242204519639 + +(8, 15) + + + +-4564177524699932913->-6847281242204519639 + + + + + +-6847281242204519639->1139456137791624401 + + + + + +-6847282503403848560 + +(9, 15) + + + +1258833679108488270->-6847282503403848560 + + + + + +-6847282503403848560->1139456137791624401 + + + + + +-6847283764603177481 + +(10, 15) + + + +-2043525817259567667->-6847283764603177481 + + + + + +-6847283764603177481->1139456137791624401 + + + + + +-6847285025802506402 + +(11, 15) + + + +1842194514200053612->-6847285025802506402 + + + + + +-6847285025802506402->1139456137791624401 + + + + + +-6847286287001835323 + +(12, 15) + + + +3612503031959884507->-6847286287001835323 + + + + + +-6847286287001835323->1139456137791624401 + + + + + +-6847287548201164244 + +(13, 15) + + + +7020363470148638746->-6847287548201164244 + + + + + +-6847287548201164244->1139456137791624401 + + + + + +-6847288809400493165 + +(14, 15) + + + +8651971472937736441->-6847288809400493165 + + + + + +-6847288809400493165->1139456137791624401 + + + + + +-6847290070599822086 + +(15, 15) + + + +2628962829197734248->-6847290070599822086 + + + + + +-6847290070599822086->1139456137791624401 + + + + + +-6847271152647169839 + +(16, 15) + + + +-8694851353339872777->-6847271152647169839 + + + + + +-6847271152647169839->1139456137791624401 + + + + + +-6847272413846498760 + +(17, 15) + + + +-4312889269536894250->-6847272413846498760 + + + + + +-6847272413846498760->1139456137791624401 + + + + + +-6847273675045827681 + +(18, 15) + + + +-6404035595180120043->-6847273675045827681 + + + + + +-6847273675045827681->1139456137791624401 + + + + + +-6847274936245156602 + +(19, 15) + + + +-8944023141916414636->-6847274936245156602 + + + + + +-6847274936245156602->1139456137791624401 + + + + + +-6847276197444485523 + +(20, 15) + + + +8944382277282004419->-6847276197444485523 + + + + + +-6847276197444485523->1139456137791624401 + + + + + +-6847277458643814444 + +(21, 15) + + + +-2032915770529528158->-6847277458643814444 + + + + + +-6847277458643814444->1139456137791624401 + + + + + +-6847278719843143365 + +(22, 15) + + + +6959449234391849953->-6847278719843143365 + + + + + +-6847278719843143365->1139456137791624401 + + + + + +-6847279981005190718 + +(23, 15) + + + +-2634912434131576464->-6847279981005190718 + + + + + +-6847279981005190718->1139456137791624401 + + + + + +-6847261063052538471 + +(24, 15) + + + +5634931922622268831->-6847261063052538471 + + + + + +-6847261063052538471->1139456137791624401 + + + + + +-6847262324251867392 + +(25, 15) + + + +-7582961617179789602->-6847262324251867392 + + + + + +-6847262324251867392->1139456137791624401 + + + + + +-6847263585451196313 + +(26, 15) + + + +-6451836927807521507->-6847263585451196313 + + + + + +-6847263585451196313->1139456137791624401 + + + + + +-6847264846650525234 + +(27, 15) + + + +-394030507489185284->-6847264846650525234 + + + + + +-6847264846650525234->1139456137791624401 + + + + + +-6847266107849854155 + +(28, 15) + + + +107172274133429611->-6847266107849854155 + + + + + +-6847266107849854155->1139456137791624401 + + + + + +-6847267369049183076 + +(29, 15) + + + +-4207830947036433750->-6847267369049183076 + + + + + +-6847267369049183076->1139456137791624401 + + + + + +-6847268630248511997 + +(30, 15) + + + +3490113550971400265->-6847268630248511997 + + + + + +-6847268630248511997->1139456137791624401 + + + + + +-6847269891447840918 + +(31, 15) + + + +-1201615709588672328->-6847269891447840918 + + + + + +-6847269891447840918->1139456137791624401 + + + + + +-6847250973495188671 + +(32, 15) + + + +5148489704822366343->-6847250973495188671 + + + + + +-6847250973495188671->1139456137791624401 + + + + + +-6847252234694517592 + +(33, 15) + + + +-9139017141363440090->-6847252234694517592 + + + + + +-6847252234694517592->1139456137791624401 + + + + + +-6847253495893846513 + +(34, 15) + + + +6337473968175777445->-6847253495893846513 + + + + + +-6847253495893846513->1139456137791624401 + + + + + +-6847254757093175434 + +(35, 15) + + + +-6852959276414411548->-6847254757093175434 + + + + + +-6847254757093175434->1139456137791624401 + + + + + +-6847256018292504355 + +(36, 15) + + + +-8069232908312795053->-6847256018292504355 + + + + + +-6847256018292504355->1139456137791624401 + + + + + +-6847257279491833276 + +(37, 15) + + + +-8793198059513939470->-6847257279491833276 + + + + + +-6847257279491833276->1139456137791624401 + + + + + +-6847258540691162197 + +(38, 15) + + + +7841281703797198641->-6847258540691162197 + + + + + +-6847258540691162197->1139456137791624401 + + + + + +-6847259801853209550 + +(39, 15) + + + +7512306250801369280->-6847259801853209550 + + + + + +-6847259801853209550->1139456137791624401 + + + + + +-6847240883900557303 + +(40, 15) + + + +-869897051836207889->-6847240883900557303 + + + + + +-6847240883900557303->1139456137791624401 + + + + + +-6847242145099886224 + +(41, 15) + + + +4949636904311317294->-6847242145099886224 + + + + + +-6847242145099886224->1139456137791624401 + + + + + +-6847243406299215145 + +(42, 15) + + + +-2730495331734781011->-6847243406299215145 + + + + + +-6847243406299215145->1139456137791624401 + + + + + +-6847244667498544066 + +(43, 15) + + + +-6334242460535207604->-6847244667498544066 + + + + + +-6847244667498544066->1139456137791624401 + + + + + +-6847245928697872987 + +(44, 15) + + + +-5951020374175440709->-6847245928697872987 + + + + + +-6847245928697872987->1139456137791624401 + + + + + +-6847247189897201908 + +(45, 15) + + + +-2468768178378955270->-6847247189897201908 + + + + + +-6847247189897201908->1139456137791624401 + + + + + +-6847248451096530829 + +(46, 15) + + + +-2456565772629031975->-6847248451096530829 + + + + + +-6847248451096530829->1139456137791624401 + + + + + +-6847249712295859750 + +(47, 15) + + + +-5542142251584758968->-6847249712295859750 + + + + + +-6847249712295859750->1139456137791624401 + + + + + +-6847230794343207503 + +(48, 15) + + + +-9160047205025049641->-6847230794343207503 + + + + + +-6847230794343207503->1139456137791624401 + + + + + +-6847291331814296644 + +(0, 16) + + + +-4666122521912370998->-6847291331814296644 + + + + + +4462707218239603676 + +simple-shuffle-combine + + + +-6847291331814296644->4462707218239603676 + + + + + +-6847292593011295467 + +(1, 16) + + + +-3650015331882301365->-6847292593011295467 + + + + + +-6847292593011295467->4462707218239603676 + + + + + +-6847293854208294290 + +(2, 16) + + + +5776795197444707996->-6847293854208294290 + + + + + +-6847293854208294290->4462707218239603676 + + + + + +-6847295115405293113 + +(3, 16) + + + +-1363796001833846147->-6847295115405293113 + + + + + +-6847295115405293113->4462707218239603676 + + + + + +-6847296376602291936 + +(4, 16) + + + +-3530337333358067202->-6847296376602291936 + + + + + +-6847296376602291936->4462707218239603676 + + + + + +-6847297637799290759 + +(5, 16) + + + +-1384098900173035905->-6847297637799290759 + + + + + +-6847297637799290759->4462707218239603676 + + + + + +-6847298898996289582 + +(6, 16) + + + +4971332213787231840->-6847298898996289582 + + + + + +-6847298898996289582->4462707218239603676 + + + + + +-6847300160156006837 + +(7, 16) + + + +-6854852325354253871->-6847300160156006837 + + + + + +-6847300160156006837->4462707218239603676 + + + + + +-6847281242238306060 + +(8, 16) + + + +-71156936529995070->-6847281242238306060 + + + + + +-6847281242238306060->4462707218239603676 + + + + + +-6847282503435304883 + +(9, 16) + + + +1923945623603648163->-6847282503435304883 + + + + + +-6847282503435304883->4462707218239603676 + + + + + +-6847283764632303706 + +(10, 16) + + + +-3978427634718832652->-6847283764632303706 + + + + + +-6847283764632303706->4462707218239603676 + + + + + +-6847285025829302529 + +(11, 16) + + + +-1799284193455936395->-6847285025829302529 + + + + + +-6847285025829302529->4462707218239603676 + + + + + +-6847286287026301352 + +(12, 16) + + + +-4329487973330979850->-6847286287026301352 + + + + + +-6847286287026301352->4462707218239603676 + + + + + +-6847287548223300175 + +(13, 16) + + + +-3560105438851128361->-6847287548223300175 + + + + + +-6847287548223300175->4462707218239603676 + + + + + +-6847288809420298998 + +(14, 16) + + + +-4369877692225896616->-6847288809420298998 + + + + + +-6847288809420298998->4462707218239603676 + + + + + +-6847290070617297821 + +(15, 16) + + + +-2093930094953632343->-6847290070617297821 + + + + + +-6847290070617297821->4462707218239603676 + + + + + +-6847271152625033908 + +(16, 16) + + + +99806127217398330->-6847271152625033908 + + + + + +-6847271152625033908->4462707218239603676 + + + + + +-6847272413822032731 + +(17, 16) + + + +3534454525792805307->-6847272413822032731 + + + + + +-6847272413822032731->4462707218239603676 + + + + + +-6847273675019031554 + +(18, 16) + + + +8424693806398724108->-6847273675019031554 + + + + + +-6847273675019031554->4462707218239603676 + + + + + +-6847274936216030377 + +(19, 16) + + + +2871648071461419821->-6847274936216030377 + + + + + +-6847274936216030377->4462707218239603676 + + + + + +-6847276197413029200 + +(20, 16) + + + +-152168639420908690->-6847276197413029200 + + + + + +-6847276197413029200->4462707218239603676 + + + + + +-6847277458610028023 + +(21, 16) + + + +-6729441186959760657->-6847277458610028023 + + + + + +-6847277458610028023->4462707218239603676 + + + + + +-6847278719807026846 + +(22, 16) + + + +1194932699260262160->-6847278719807026846 + + + + + +-6847278719807026846->4462707218239603676 + + + + + +-6847279981041307237 + +(23, 16) + + + +1260190108635870785->-6847279981041307237 + + + + + +-6847279981041307237->4462707218239603676 + + + + + +-6847261063049043324 + +(24, 16) + + + +3393644815770713906->-6847261063049043324 + + + + + +-6847261063049043324->4462707218239603676 + + + + + +-6847262324246042147 + +(25, 16) + + + +4336196443053145939->-6847262324246042147 + + + + + +-6847262324246042147->4462707218239603676 + + + + + +-6847263585443040970 + +(26, 16) + + + +2373597681405778084->-6847263585443040970 + + + + + +-6847263585443040970->4462707218239603676 + + + + + +-6847264846640039793 + +(27, 16) + + + +4113469716384568357->-6847264846640039793 + + + + + +-6847264846640039793->4462707218239603676 + + + + + +-6847266107837038616 + +(28, 16) + + + +3291227880397728102->-6847266107837038616 + + + + + +-6847266107837038616->4462707218239603676 + + + + + +-6847267369034037439 + +(29, 16) + + + +-4139928299350252409->-6847267369034037439 + + + + + +-6847267369034037439->4462707218239603676 + + + + + +-6847268630231036262 + +(30, 16) + + + +535520570058907656->-6847268630231036262 + + + + + +-6847268630231036262->4462707218239603676 + + + + + +-6847269891428035085 + +(31, 16) + + + +2009830661114817113->-6847269891428035085 + + + + + +-6847269891428035085->4462707218239603676 + + + + + +-6847250973510334308 + +(32, 16) + + + +4422422658414751402->-6847250973510334308 + + + + + +-6847250973510334308->4462707218239603676 + + + + + +-6847252234707333131 + +(33, 16) + + + +4993832294432718635->-6847252234707333131 + + + + + +-6847252234707333131->4462707218239603676 + + + + + +-6847253495904331954 + +(34, 16) + + + +7784343723896759420->-6847253495904331954 + + + + + +-6847253495904331954->4462707218239603676 + + + + + +-6847254757101330777 + +(35, 16) + + + +2786367494190378077->-6847254757101330777 + + + + + +-6847254757101330777->4462707218239603676 + + + + + +-6847256018298329600 + +(36, 16) + + + +1315417425045856222->-6847256018298329600 + + + + + +-6847256018298329600->4462707218239603676 + + + + + +-6847257279495328423 + +(37, 16) + + + +3647494264493916255->-6847257279495328423 + + + + + +-6847257279495328423->4462707218239603676 + + + + + +-6847258540692327246 + +(38, 16) + + + +-8371835006123505600->-6847258540692327246 + + + + + +-6847258540692327246->4462707218239603676 + + + + + +-6847259801852044501 + +(39, 16) + + + +5456817685685366705->-6847259801852044501 + + + + + +-6847259801852044501->4462707218239603676 + + + + + +-6847240883934343724 + +(40, 16) + + + +-4784801422360516446->-6847240883934343724 + + + + + +-6847240883934343724->4462707218239603676 + + + + + +-6847242145131342547 + +(41, 16) + + + +-3135611688341330813->-6847242145131342547 + + + + + +-6847242145131342547->4462707218239603676 + + + + + +-6847243406328341370 + +(42, 16) + + + +-3942248176917517868->-6847243406328341370 + + + + + +-6847243406328341370->4462707218239603676 + + + + + +-6847244667525340193 + +(43, 16) + + + +7171616034675960405->-6847244667525340193 + + + + + +-6847244667525340193->4462707218239603676 + + + + + +-6847245928722339016 + +(44, 16) + + + +-4819593501659243050->-6847245928722339016 + + + + + +-6847245928722339016->4462707218239603676 + + + + + +-6847247189919337839 + +(45, 16) + + + +3852760803058918839->-6847247189919337839 + + + + + +-6847247189919337839->4462707218239603676 + + + + + +-6847248451116336662 + +(46, 16) + + + +2886189140323747384->-6847248451116336662 + + + + + +-6847248451116336662->4462707218239603676 + + + + + +-6847249712313335485 + +(47, 16) + + + +8133963301538507657->-6847249712313335485 + + + + + +-6847249712313335485->4462707218239603676 + + + + + +-6847230794321071572 + +(48, 16) + + + +-7664362077485643750->-6847230794321071572 + + + + + +-6847230794321071572->4462707218239603676 + + + + + +-6847291331815461693 + +(0, 17) + + + +-2916801110675116023->-6847291331815461693 + + + + + +4373067005400823351 + +simple-shuffle-combine + + + +-6847291331815461693->4373067005400823351 + + + + + +-6847292593010130418 + +(1, 17) + + + +-3459926936677971140->-6847292593010130418 + + + + + +-6847292593010130418->4373067005400823351 + + + + + +-6847293854209459339 + +(2, 17) + + + +6851346204147859371->-6847293854209459339 + + + + + +-6847293854209459339->4373067005400823351 + + + + + +-6847295115404128064 + +(3, 17) + + + +-3607404028595789922->-6847295115404128064 + + + + + +-6847295115404128064->4373067005400823351 + + + + + +-6847296376603456985 + +(4, 17) + + + +-350113698468456227->-6847296376603456985 + + + + + +-6847296376603456985->4373067005400823351 + + + + + +-6847297637798125710 + +(5, 17) + + + +-2431902900566881280->-6847297637798125710 + + + + + +-6847297637798125710->4373067005400823351 + + + + + +-6847298898997454631 + +(6, 17) + + + +-3573785981968170785->-6847298898997454631 + + + + + +-6847298898997454631->4373067005400823351 + + + + + +-6847300160154841788 + +(7, 17) + + + +-7898226314995224846->-6847300160154841788 + + + + + +-6847300160154841788->4373067005400823351 + + + + + +-6847281242239471109 + +(8, 17) + + + +-8107156654259699295->-6847281242239471109 + + + + + +-6847281242239471109->4373067005400823351 + + + + + +-6847282503434139834 + +(9, 17) + + + +-7751340781854472812->-6847282503434139834 + + + + + +-6847282503434139834->4373067005400823351 + + + + + +-6847283764633468755 + +(10, 17) + + + +5842185448930341123->-6847283764633468755 + + + + + +-6847283764633468755->4373067005400823351 + + + + + +-6847285025828137480 + +(11, 17) + + + +-2138656212072600170->-6847285025828137480 + + + + + +-6847285025828137480->4373067005400823351 + + + + + +-6847286287027466401 + +(12, 17) + + + +-3898789289905535275->-6847286287027466401 + + + + + +-6847286287027466401->4373067005400823351 + + + + + +-6847287548222135126 + +(13, 17) + + + +3958892326573458680->-6847287548222135126 + + + + + +-6847287548222135126->4373067005400823351 + + + + + +-6847288809421464047 + +(14, 17) + + + +-3335361538048349641->-6847288809421464047 + + + + + +-6847288809421464047->4373067005400823351 + + + + + +-6847290070616132772 + +(15, 17) + + + +4614566722004491882->-6847290070616132772 + + + + + +-6847290070616132772->4373067005400823351 + + + + + +-6847271152626198957 + +(16, 17) + + + +3286944390613790905->-6847271152626198957 + + + + + +-6847271152626198957->4373067005400823351 + + + + + +-6847272413820867682 + +(17, 17) + + + +-6875149715532537684->-6847272413820867682 + + + + + +-6847272413820867682->4373067005400823351 + + + + + +-6847273675020196603 + +(18, 17) + + + +-8978408514002600933->-6847273675020196603 + + + + + +-6847273675020196603->4373067005400823351 + + + + + +-6847274936214865328 + +(19, 17) + + + +707830650176076046->-6847274936214865328 + + + + + +-6847274936214865328->4373067005400823351 + + + + + +-6847276197414194249 + +(20, 17) + + + +1385493021076723085->-6847276197414194249 + + + + + +-6847276197414194249->4373067005400823351 + + + + + +-6847277458608862974 + +(21, 17) + + + +1107228292013339568->-6847277458608862974 + + + + + +-6847277458608862974->4373067005400823351 + + + + + +-6847278719808191895 + +(22, 17) + + + +-7506527031628162481->-6847278719808191895 + + + + + +-6847278719808191895->4373067005400823351 + + + + + +-6847279981040142188 + +(23, 17) + + + +912758783192327010->-6847279981040142188 + + + + + +-6847279981040142188->4373067005400823351 + + + + + +-6847261063050208373 + +(24, 17) + + + +3790744854084964881->-6847261063050208373 + + + + + +-6847261063050208373->4373067005400823351 + + + + + +-6847262324244877098 + +(25, 17) + + + +3282822238651946564->-6847262324244877098 + + + + + +-6847262324244877098->4373067005400823351 + + + + + +-6847263585444206019 + +(26, 17) + + + +3354895544766875059->-6847263585444206019 + + + + + +-6847263585444206019->4373067005400823351 + + + + + +-6847264846638874744 + +(27, 17) + + + +3065648989715083782->-6847264846638874744 + + + + + +-6847264846638874744->4373067005400823351 + + + + + +-6847266107838203665 + +(28, 17) + + + +6392632036094719877->-6847266107838203665 + + + + + +-6847266107838203665->4373067005400823351 + + + + + +-6847267369032872390 + +(29, 17) + + + +-5681027611896592984->-6847267369032872390 + + + + + +-6847267369032872390->4373067005400823351 + + + + + +-6847268630232201311 + +(30, 17) + + + +389755239205209831->-6847268630232201311 + + + + + +-6847268630232201311->4373067005400823351 + + + + + +-6847269891426870036 + +(31, 17) + + + +-8926810899166234662->-6847269891426870036 + + + + + +-6847269891426870036->4373067005400823351 + + + + + +-6847250973511499357 + +(32, 17) + + + +5944239862976703977->-6847250973511499357 + + + + + +-6847250973511499357->4373067005400823351 + + + + + +-6847252234706168082 + +(33, 17) + + + +3940495538676204060->-6847252234706168082 + + + + + +-6847252234706168082->4373067005400823351 + + + + + +-6847253495905497003 + +(34, 17) + + + +8761212649573043595->-6847253495905497003 + + + + + +-6847253495905497003->4373067005400823351 + + + + + +-6847254757100165728 + +(35, 17) + + + +-8875546579365174914->-6847254757100165728 + + + + + +-6847254757100165728->4373067005400823351 + + + + + +-6847256018299494649 + +(36, 17) + + + +1731785579752171197->-6847256018299494649 + + + + + +-6847256018299494649->4373067005400823351 + + + + + +-6847257279494163374 + +(37, 17) + + + +-6253082962413285920->-6847257279494163374 + + + + + +-6847257279494163374->4373067005400823351 + + + + + +-6847258540693492295 + +(38, 17) + + + +1457802644476854975->-6847258540693492295 + + + + + +-6847258540693492295->4373067005400823351 + + + + + +-6847259801850879452 + +(39, 17) + + + +5611478323925790930->-6847259801850879452 + + + + + +-6847259801850879452->4373067005400823351 + + + + + +-6847240883935508773 + +(40, 17) + + + +5031381662781305729->-6847240883935508773 + + + + + +-6847240883935508773->4373067005400823351 + + + + + +-6847242145130177498 + +(41, 17) + + + +5412831580863793012->-6847242145130177498 + + + + + +-6847242145130177498->4373067005400823351 + + + + + +-6847243406329506419 + +(42, 17) + + + +6026987645718724323->-6847243406329506419 + + + + + +-6847243406329506419->4373067005400823351 + + + + + +-6847244667524175144 + +(43, 17) + + + +-4313313246320786570->-6847244667524175144 + + + + + +-6847244667524175144->4373067005400823351 + + + + + +-6847245928723504065 + +(44, 17) + + + +-2039246511588652875->-6847245928723504065 + + + + + +-6847245928723504065->4373067005400823351 + + + + + +-6847247189918172790 + +(45, 17) + + + +4070570694097167064->-6847247189918172790 + + + + + +-6847247189918172790->4373067005400823351 + + + + + +-6847248451117501711 + +(46, 17) + + + +4003953683735367959->-6847248451117501711 + + + + + +-6847248451117501711->4373067005400823351 + + + + + +-6847249712312170436 + +(47, 17) + + + +5406070676528811082->-6847249712312170436 + + + + + +-6847249712312170436->4373067005400823351 + + + + + +-6847230794322236621 + +(48, 17) + + + +2826141082157567641->-6847230794322236621 + + + + + +-6847230794322236621->4373067005400823351 + + + + + +-6847291331811966546 + +(0, 18) + + + +-7206269399868576548->-6847291331811966546 + + + + + +5395749863437186626 + +simple-shuffle-combine + + + +-6847291331811966546->5395749863437186626 + + + + + +-6847292593013625565 + +(1, 18) + + + +133565742423124585->-6847292593013625565 + + + + + +-6847292593013625565->5395749863437186626 + + + + + +-6847293854210624388 + +(2, 18) + + + +-8920298307025796470->-6847293854210624388 + + + + + +-6847293854210624388->5395749863437186626 + + + + + +-6847295115402963015 + +(3, 18) + + + +-4584271820288165697->-6847295115402963015 + + + + + +-6847295115402963015->5395749863437186626 + + + + + +-6847296376599961838 + +(4, 18) + + + +3457890104974041248->-6847296376599961838 + + + + + +-6847296376599961838->5395749863437186626 + + + + + +-6847297637801620857 + +(5, 18) + + + +-7293376678730802371->-6847297637801620857 + + + + + +-6847297637801620857->5395749863437186626 + + + + + +-6847298898998619680 + +(6, 18) + + + +-845893320343148610->-6847298898998619680 + + + + + +-6847298898998619680->5395749863437186626 + + + + + +-6847300160153676739 + +(7, 18) + + + +6232101836192463795->-6847300160153676739 + + + + + +-6847300160153676739->5395749863437186626 + + + + + +-6847281242235975962 + +(8, 18) + + + +-4317989564525065420->-6847281242235975962 + + + + + +-6847281242235975962->5395749863437186626 + + + + + +-6847282503437634981 + +(9, 18) + + + +-4986961968399474687->-6847282503437634981 + + + + + +-6847282503437634981->5395749863437186626 + + + + + +-6847283764634633804 + +(10, 18) + + + +6274789343135258498->-6847283764634633804 + + + + + +-6847283764634633804->5395749863437186626 + + + + + +-6847285025826972431 + +(11, 18) + + + +-3183139705003251945->-6847285025826972431 + + + + + +-6847285025826972431->5395749863437186626 + + + + + +-6847286287023971254 + +(12, 18) + + + +1449080235178853016->-6847286287023971254 + + + + + +-6847286287023971254->5395749863437186626 + + + + + +-6847287548225630273 + +(13, 18) + + + +9093106721763076405->-6847287548225630273 + + + + + +-6847287548225630273->5395749863437186626 + + + + + +-6847288809422629096 + +(14, 18) + + + +-3058901591902670666->-6847288809422629096 + + + + + +-6847288809422629096->5395749863437186626 + + + + + +-6847290070614967723 + +(15, 18) + + + +3570084337614080907->-6847290070614967723 + + + + + +-6847290070614967723->5395749863437186626 + + + + + +-6847271152622703810 + +(16, 18) + + + +7120603515298850380->-6847271152622703810 + + + + + +-6847271152622703810->5395749863437186626 + + + + + +-6847272413824362829 + +(17, 18) + + + +7815780101328268057->-6847272413824362829 + + + + + +-6847272413824362829->5395749863437186626 + + + + + +-6847273675021361652 + +(18, 18) + + + +2645316713263944442->-6847273675021361652 + + + + + +-6847273675021361652->5395749863437186626 + + + + + +-6847274936213700279 + +(19, 18) + + + +7624917281835082287->-6847274936213700279 + + + + + +-6847274936213700279->5395749863437186626 + + + + + +-6847276197410699102 + +(20, 18) + + + +8116723519700234576->-6847276197410699102 + + + + + +-6847276197410699102->5395749863437186626 + + + + + +-6847277458612358121 + +(21, 18) + + + +-3911543324150981651->-6847277458612358121 + + + + + +-6847277458612358121->5395749863437186626 + + + + + +-6847278719809356944 + +(22, 18) + + + +-5938850860570615506->-6847278719809356944 + + + + + +-6847278719809356944->5395749863437186626 + + + + + +-6847279981038977139 + +(23, 18) + + + +-471377490525160221->-6847279981038977139 + + + + + +-6847279981038977139->5395749863437186626 + + + + + +-6847261063046713226 + +(24, 18) + + + +-390241409878844444->-6847261063046713226 + + + + + +-6847261063046713226->5395749863437186626 + + + + + +-6847262324248372245 + +(25, 18) + + + +6911779000428139889->-6847262324248372245 + + + + + +-6847262324248372245->5395749863437186626 + + + + + +-6847263585445371068 + +(26, 18) + + + +4911789705267720434->-6847263585445371068 + + + + + +-6847263585445371068->5395749863437186626 + + + + + +-6847264846637709695 + +(27, 18) + + + +2139915399671480007->-6847264846637709695 + + + + + +-6847264846637709695->5395749863437186626 + + + + + +-6847266107834708518 + +(28, 18) + + + +8992673751426318152->-6847266107834708518 + + + + + +-6847266107834708518->5395749863437186626 + + + + + +-6847267369036367537 + +(29, 18) + + + +8419413684591917541->-6847267369036367537 + + + + + +-6847267369036367537->5395749863437186626 + + + + + +-6847268630233366360 + +(30, 18) + + + +3050037874032122406->-6847268630233366360 + + + + + +-6847268630233366360->5395749863437186626 + + + + + +-6847269891425704987 + +(31, 18) + + + +8485454434646781179->-6847269891425704987 + + + + + +-6847269891425704987->5395749863437186626 + + + + + +-6847250973508004210 + +(32, 18) + + + +1951109928847589052->-6847250973508004210 + + + + + +-6847250973508004210->5395749863437186626 + + + + + +-6847252234709663229 + +(33, 18) + + + +9213252010927885385->-6847252234709663229 + + + + + +-6847252234709663229->5395749863437186626 + + + + + +-6847253495906662052 + +(34, 18) + + + +-5298181573109194646->-6847253495906662052 + + + + + +-6847253495906662052->5395749863437186626 + + + + + +-6847254757099000679 + +(35, 18) + + + +-175195243260831073->-6847254757099000679 + + + + + +-6847254757099000679->5395749863437186626 + + + + + +-6847256018295999502 + +(36, 18) + + + +-1150625843348633728->-6847256018295999502 + + + + + +-6847256018295999502->5395749863437186626 + + + + + +-6847257279497658521 + +(37, 18) + + + +-4434004478321012195->-6847257279497658521 + + + + + +-6847257279497658521->5395749863437186626 + + + + + +-6847258540694657344 + +(38, 18) + + + +2994343788647029150->-6847258540694657344 + + + + + +-6847258540694657344->5395749863437186626 + + + + + +-6847259801849714403 + +(39, 18) + + + +4639039389031959955->-6847259801849714403 + + + + + +-6847259801849714403->5395749863437186626 + + + + + +-6847240883932013626 + +(40, 18) + + + +1008747192606026004->-6847240883932013626 + + + + + +-6847240883932013626->5395749863437186626 + + + + + +-6847242145133672645 + +(41, 18) + + + +8230529654459002337->-6847242145133672645 + + + + + +-6847242145133672645->5395749863437186626 + + + + + +-6847243406330671468 + +(42, 18) + + + +6515124041236307298->-6847243406330671468 + + + + + +-6847243406330671468->5395749863437186626 + + + + + +-6847244667523010095 + +(43, 18) + + + +4156444414466270455->-6847244667523010095 + + + + + +-6847244667523010095->5395749863437186626 + + + + + +-6847245928720008918 + +(44, 18) + + + +2183454263025377400->-6847245928720008918 + + + + + +-6847245928720008918->5395749863437186626 + + + + + +-6847247189921667937 + +(45, 18) + + + +9138181466998016789->-6847247189921667937 + + + + + +-6847247189921667937->5395749863437186626 + + + + + +-6847248451118666760 + +(46, 18) + + + +7110873859703306134->-6847248451118666760 + + + + + +-6847248451118666760->5395749863437186626 + + + + + +-6847249712311005387 + +(47, 18) + + + +3988098037103008107->-6847249712311005387 + + + + + +-6847249712311005387->5395749863437186626 + + + + + +-6847230794318741474 + +(48, 18) + + + +-2911076095335651284->-6847230794318741474 + + + + + +-6847230794318741474->5395749863437186626 + + + + + +-6847291331813131595 + +(0, 19) + + + +-5639669815009597973->-6847291331813131595 + + + + + +-3025610838906466899 + +simple-shuffle-combine + + + +-6847291331813131595->-3025610838906466899 + + + + + +-6847292593012460516 + +(1, 19) + + + +-2606641376605003990->-6847292593012460516 + + + + + +-6847292593012460516->-3025610838906466899 + + + + + +-6847293854211789437 + +(2, 19) + + + +-8615837915483575095->-6847293854211789437 + + + + + +-6847293854211789437->-3025610838906466899 + + + + + +-6847295115401797966 + +(3, 19) + + + +4441541530634315328->-6847295115401797966 + + + + + +-6847295115401797966->-3025610838906466899 + + + + + +-6847296376601126887 + +(4, 19) + + + +-4574820826254078177->-6847296376601126887 + + + + + +-6847296376601126887->-3025610838906466899 + + + + + +-6847297637800455808 + +(5, 19) + + + +-410283888839962530->-6847297637800455808 + + + + + +-6847297637800455808->-3025610838906466899 + + + + + +-6847298898999784729 + +(6, 19) + + + +-8348227419910455651->-6847298898999784729 + + + + + +-6847298898999784729->-3025610838906466899 + + + + + +-6847300160152511690 + +(7, 19) + + + +5188723305021370020->-6847300160152511690 + + + + + +-6847300160152511690->-3025610838906466899 + + + + + +-6847281242237141011 + +(8, 19) + + + +7364721822182498371->-6847281242237141011 + + + + + +-6847281242237141011->-3025610838906466899 + + + + + +-6847282503436469932 + +(9, 19) + + + +3491654631957224738->-6847282503436469932 + + + + + +-6847282503436469932->-3025610838906466899 + + + + + +-6847283764635798853 + +(10, 19) + + + +7247228270304191073->-6847283764635798853 + + + + + +-6847283764635798853->-3025610838906466899 + + + + + +-6847285025825807382 + +(11, 19) + + + +2504770552126039096->-6847285025825807382 + + + + + +-6847285025825807382->-3025610838906466899 + + + + + +-6847286287025136303 + +(12, 19) + + + +-5301895151825663625->-6847286287025136303 + + + + + +-6847286287025136303->-3025610838906466899 + + + + + +-6847287548224465224 + +(13, 19) + + + +-2516726907628073386->-6847287548224465224 + + + + + +-6847287548224465224->-3025610838906466899 + + + + + +-6847288809423794145 + +(14, 19) + + + +8103926044674373525->-6847288809423794145 + + + + + +-6847288809423794145->-3025610838906466899 + + + + + +-6847290070613802674 + +(15, 19) + + + +1691948279327697532->-6847290070613802674 + + + + + +-6847290070613802674->-3025610838906466899 + + + + + +-6847271152623868859 + +(16, 19) + + + +-2055094756512686245->-6847271152623868859 + + + + + +-6847271152623868859->-3025610838906466899 + + + + + +-6847272413823197780 + +(17, 19) + + + +-3839217702092991334->-6847272413823197780 + + + + + +-6847272413823197780->-3025610838906466899 + + + + + +-6847273675022526701 + +(18, 19) + + + +3689800198435057017->-6847273675022526701 + + + + + +-6847273675022526701->-3025610838906466899 + + + + + +-6847274936212535230 + +(19, 19) + + + +7770682623808476912->-6847274936212535230 + + + + + +-6847274936212535230->-3025610838906466899 + + + + + +-6847276197411864151 + +(20, 19) + + + +8448323692453086351->-6847276197411864151 + + + + + +-6847276197411864151->-3025610838906466899 + + + + + +-6847277458611193072 + +(21, 19) + + + +-6586660714851723826->-6847277458611193072 + + + + + +-6847277458611193072->-3025610838906466899 + + + + + +-6847278719810521993 + +(22, 19) + + + +-4895459036113764531->-6847278719810521993 + + + + + +-6847278719810521993->-3025610838906466899 + + + + + +-6847279981037812090 + +(23, 19) + + + +-2094190472936087596->-6847279981037812090 + + + + + +-6847279981037812090->-3025610838906466899 + + + + + +-6847261063047878275 + +(24, 19) + + + +2275611446760793331->-6847261063047878275 + + + + + +-6847261063047878275->-3025610838906466899 + + + + + +-6847262324247207196 + +(25, 19) + + + +5382989607097620114->-6847262324247207196 + + + + + +-6847262324247207196->-3025610838906466899 + + + + + +-6847263585446536117 + +(26, 19) + + + +6316441590306118609->-6847263585446536117 + + + + + +-6847263585446536117->-3025610838906466899 + + + + + +-6847264846636544646 + +(27, 19) + + + +-8399487272735292184->-6847264846636544646 + + + + + +-6847264846636544646->-3025610838906466899 + + + + + +-6847266107835873567 + +(28, 19) + + + +2216717736553829927->-6847266107835873567 + + + + + +-6847266107835873567->-3025610838906466899 + + + + + +-6847267369035202488 + +(29, 19) + + + +-3092124228081330234->-6847267369035202488 + + + + + +-6847267369035202488->-3025610838906466899 + + + + + +-6847268630234531409 + +(30, 19) + + + +-4359083946074287035->-6847268630234531409 + + + + + +-6847268630234531409->-3025610838906466899 + + + + + +-6847269891424539938 + +(31, 19) + + + +-9102645481964936212->-6847269891424539938 + + + + + +-6847269891424539938->-3025610838906466899 + + + + + +-6847250973509169259 + +(32, 19) + + + +2933516149268935627->-6847250973509169259 + + + + + +-6847250973509169259->-3025610838906466899 + + + + + +-6847252234708498180 + +(33, 19) + + + +8169572800124249610->-6847252234708498180 + + + + + +-6847252234708498180->-3025610838906466899 + + + + + +-6847253495907827101 + +(34, 19) + + + +-4325742645974902871->-6847253495907827101 + + + + + +-6847253495907827101->-3025610838906466899 + + + + + +-6847254757097835630 + +(35, 19) + + + +-2985230133043911648->-6847254757097835630 + + + + + +-6847254757097835630->-3025610838906466899 + + + + + +-6847256018297164551 + +(36, 19) + + + +8603517437968270847->-6847256018297164551 + + + + + +-6847256018297164551->-3025610838906466899 + + + + + +-6847257279496493472 + +(37, 19) + + + +4052381196600858430->-6847257279496493472 + + + + + +-6847257279496493472->-3025610838906466899 + + + + + +-6847258540695822393 + +(38, 19) + + + +-5562978043950412675->-6847258540695822393 + + + + + +-6847258540695822393->-3025610838906466899 + + + + + +-6847259801848549354 + +(39, 19) + + + +251685005342568580->-6847259801848549354 + + + + + +-6847259801848549354->-3025610838906466899 + + + + + +-6847240883933178675 + +(40, 19) + + + +2052089408025158179->-6847240883933178675 + + + + + +-6847240883933178675->-3025610838906466899 + + + + + +-6847242145132507596 + +(41, 19) + + + +-1515826767394243838->-6847242145132507596 + + + + + +-6847242145132507596->-3025610838906466899 + + + + + +-6847243406331836517 + +(42, 19) + + + +-2114331995600375743->-6847243406331836517 + + + + + +-6847243406331836517->-3025610838906466899 + + + + + +-6847244667521845046 + +(43, 19) + + + +2615345101919929880->-6847244667521845046 + + + + + +-6847244667521845046->-3025610838906466899 + + + + + +-6847245928721173967 + +(44, 19) + + + +2036580565134879575->-6847245928721173967 + + + + + +-6847245928721173967->-3025610838906466899 + + + + + +-6847247189920502888 + +(45, 19) + + + +7049866361942057014->-6847247189920502888 + + + + + +-6847247189920502888->-3025610838906466899 + + + + + +-6847248451119831809 + +(46, 19) + + + +8154265755035233909->-6847248451119831809 + + + + + +-6847248451119831809->-3025610838906466899 + + + + + +-6847249712309840338 + +(47, 19) + + + +2422349512960957532->-6847249712309840338 + + + + + +-6847249712309840338->-3025610838906466899 + + + + + +-6847230794319906523 + +(48, 19) + + + +8619695379306495291->-6847230794319906523 + + + + + +-6847230794319906523->-3025610838906466899 + + + + + +-6847291331809636448 + +(0, 20) + + + +8265622953955227518->-6847291331809636448 + + + + + +-9196218965325156440 + +simple-shuffle-combine + + + +-6847291331809636448->-9196218965325156440 + + + + + +-6847292593006635271 + +(1, 20) + + + +-6829538520669144065->-6847292593006635271 + + + + + +-6847292593006635271->-9196218965325156440 + + + + + +-6847293854203634094 + +(2, 20) + + + +1362969017100182496->-6847293854203634094 + + + + + +-6847293854203634094->-9196218965325156440 + + + + + +-6847295115409953309 + +(3, 20) + + + +5712573200678899753->-6847295115409953309 + + + + + +-6847295115409953309->-9196218965325156440 + + + + + +-6847296376606952132 + +(4, 20) + + + +2099376246823853898->-6847296376606952132 + + + + + +-6847296376606952132->-9196218965325156440 + + + + + +-6847297637803950955 + +(5, 20) + + + +4171491891031317195->-6847297637803950955 + + + + + +-6847297637803950955->-9196218965325156440 + + + + + +-6847298899000949778 + +(6, 20) + + + +1621008368414074140->-6847298899000949778 + + + + + +-6847298899000949778->-9196218965325156440 + + + + + +-6847300160151346641 + +(7, 20) + + + +3730873507950792645->-6847300160151346641 + + + + + +-6847300160151346641->-9196218965325156440 + + + + + +-6847281242233645864 + +(8, 20) + + + +-5153537652708869770->-6847281242233645864 + + + + + +-6847281242233645864->-9196218965325156440 + + + + + +-6847282503430644687 + +(9, 20) + + + +4839411449728680279->-6847282503430644687 + + + + + +-6847282503430644687->-9196218965325156440 + + + + + +-6847283764627643510 + +(10, 20) + + + +-2317476625053574952->-6847283764627643510 + + + + + +-6847283764627643510->-9196218965325156440 + + + + + +-6847285025833962725 + +(11, 20) + + + +2084240930883034305->-6847285025833962725 + + + + + +-6847285025833962725->-9196218965325156440 + + + + + +-6847286287030961548 + +(12, 20) + + + +1265336373009891650->-6847286287030961548 + + + + + +-6847286287030961548->-9196218965325156440 + + + + + +-6847287548227960371 + +(13, 20) + + + +2082470217162523939->-6847287548227960371 + + + + + +-6847287548227960371->-9196218965325156440 + + + + + +-6847288809424959194 + +(14, 20) + + + +109516416963092084->-6847288809424959194 + + + + + +-6847288809424959194->-9196218965325156440 + + + + + +-6847290070612637625 + +(15, 20) + + + +-471847837553879043->-6847290070612637625 + + + + + +-6847290070612637625->-9196218965325156440 + + + + + +-6847271152620373712 + +(16, 20) + + + +-5565374986417553170->-6847271152620373712 + + + + + +-6847271152620373712->-9196218965325156440 + + + + + +-6847272413817372535 + +(17, 20) + + + +6421158251661802607->-6847272413817372535 + + + + + +-6847272413817372535->-9196218965325156440 + + + + + +-6847273675014371358 + +(18, 20) + + + +-6023917159247511408->-6847273675014371358 + + + + + +-6847273675014371358->-9196218965325156440 + + + + + +-6847274936220690573 + +(19, 20) + + + +7346559056942895321->-6847274936220690573 + + + + + +-6847274936220690573->-9196218965325156440 + + + + + +-6847276197417689396 + +(20, 20) + + + +7708834553089358010->-6847276197417689396 + + + + + +-6847276197417689396->-9196218965325156440 + + + + + +-6847277458614688219 + +(21, 20) + + + +387531944478602299->-6847277458614688219 + + + + + +-6847277458614688219->-9196218965325156440 + + + + + +-6847278719811687042 + +(22, 20) + + + +5086091256136466060->-6847278719811687042 + + + + + +-6847278719811687042->-9196218965325156440 + + + + + +-6847279981036647041 + +(23, 20) + + + +-1947624202169039371->-6847279981036647041 + + + + + +-6847279981036647041->-9196218965325156440 + + + + + +-6847261063044383128 + +(24, 20) + + + +-3365088579383744794->-6847261063044383128 + + + + + +-6847261063044383128->-9196218965325156440 + + + + + +-6847262324241381951 + +(25, 20) + + + +-104274645800903161->-6847262324241381951 + + + + + +-6847262324241381951->-9196218965325156440 + + + + + +-6847263585438380774 + +(26, 20) + + + +-3397008040237917816->-6847263585438380774 + + + + + +-6847263585438380774->-9196218965325156440 + + + + + +-6847264846644699989 + +(27, 20) + + + +-7072656448450718159->-6847264846644699989 + + + + + +-6847264846644699989->-9196218965325156440 + + + + + +-6847266107841698812 + +(28, 20) + + + +7704254291358855602->-6847266107841698812 + + + + + +-6847266107841698812->-9196218965325156440 + + + + + +-6847267369038697635 + +(29, 20) + + + +300506510332519891->-6847267369038697635 + + + + + +-6847267369038697635->-9196218965325156440 + + + + + +-6847268630235696458 + +(30, 20) + + + +5609043529495576356->-6847268630235696458 + + + + + +-6847268630235696458->-9196218965325156440 + + + + + +-6847269891423374889 + +(31, 20) + + + +-632887854363765587->-6847269891423374889 + + + + + +-6847269891423374889->-9196218965325156440 + + + + + +-6847250973505674112 + +(32, 20) + + + +-698881122421966498->-6847250973505674112 + + + + + +-6847250973505674112->-9196218965325156440 + + + + + +-6847252234702672935 + +(33, 20) + + + +518904536118368735->-6847252234702672935 + + + + + +-6847252234702672935->-9196218965325156440 + + + + + +-6847253495899671758 + +(34, 20) + + + +3383801941172969920->-6847253495899671758 + + + + + +-6847253495899671758->-9196218965325156440 + + + + + +-6847254757105990973 + +(35, 20) + + + +9032403566833709577->-6847254757105990973 + + + + + +-6847254757105990973->-9196218965325156440 + + + + + +-6847256018302989796 + +(36, 20) + + + +5869170301590094122->-6847256018302989796 + + + + + +-6847256018302989796->-9196218965325156440 + + + + + +-6847257279499988619 + +(37, 20) + + + +-9138497561212632661->-6847257279499988619 + + + + + +-6847257279499988619->-9196218965325156440 + + + + + +-6847258540696987442 + +(38, 20) + + + +8340829643911592700->-6847258540696987442 + + + + + +-6847258540696987442->-9196218965325156440 + + + + + +-6847259801847384305 + +(39, 20) + + + +-725182865053704795->-6847259801847384305 + + + + + +-6847259801847384305->-9196218965325156440 + + + + + +-6847240883929683528 + +(40, 20) + + + +-3123815178827442346->-6847240883929683528 + + + + + +-6847240883929683528->-9196218965325156440 + + + + + +-6847242145126682351 + +(41, 20) + + + +-981462841803047113->-6847242145126682351 + + + + + +-6847242145126682351->-9196218965325156440 + + + + + +-6847243406323681174 + +(42, 20) + + + +6568798028934046648->-6847243406323681174 + + + + + +-6847243406323681174->-9196218965325156440 + + + + + +-6847244667530000389 + +(43, 20) + + + +-4455614907987618911->-6847244667530000389 + + + + + +-6847244667530000389->-9196218965325156440 + + + + + +-6847245928726999212 + +(44, 20) + + + +1904738682600066850->-6847245928726999212 + + + + + +-6847245928726999212->-9196218965325156440 + + + + + +-6847247189923998035 + +(45, 20) + + + +1075915152220039939->-6847247189923998035 + + + + + +-6847247189923998035->-9196218965325156440 + + + + + +-6847248451120996858 + +(46, 20) + + + +7434671350715084884->-6847248451120996858 + + + + + +-6847248451120996858->-9196218965325156440 + + + + + +-6847249712308675289 + +(47, 20) + + + +1375515459401134557->-6847249712308675289 + + + + + +-6847249712308675289->-9196218965325156440 + + + + + +-6847230794316411376 + +(48, 20) + + + +5779795272075522766->-6847230794316411376 + + + + + +-6847230794316411376->-9196218965325156440 + + + + + +-6847291331810801497 + +(0, 21) + + + +-7501570253216124323->-6847291331810801497 + + + + + +-8098914186541078765 + +simple-shuffle-combine + + + +-6847291331810801497->-8098914186541078765 + + + + + +-6847292593005470222 + +(1, 21) + + + +-7872912510344755840->-6847292593005470222 + + + + + +-6847292593005470222->-8098914186541078765 + + + + + +-6847293854204799143 + +(2, 21) + + + +2262125795259096671->-6847293854204799143 + + + + + +-6847293854204799143->-8098914186541078765 + + + + + +-6847295115408788260 + +(3, 21) + + + +4363457285501830378->-6847295115408788260 + + + + + +-6847295115408788260->-8098914186541078765 + + + + + +-6847296376608117181 + +(4, 21) + + + +-6522289507553159543->-6847296376608117181 + + + + + +-6847296376608117181->-8098914186541078765 + + + + + +-6847297637802785906 + +(5, 21) + + + +2604924088136396220->-6847297637802785906 + + + + + +-6847297637802785906->-8098914186541078765 + + + + + +-6847298899002114827 + +(6, 21) + + + +4397107079729833515->-6847298899002114827 + + + + + +-6847298899002114827->-8098914186541078765 + + + + + +-6847300160150181592 + +(7, 21) + + + +3436658576584015270->-6847300160150181592 + + + + + +-6847300160150181592->-8098914186541078765 + + + + + +-6847281242234810913 + +(8, 21) + + + +-4100178531726049195->-6847281242234810913 + + + + + +-6847281242234810913->-8098914186541078765 + + + + + +-6847282503429479638 + +(9, 21) + + + +-6239170802135122312->-6847282503429479638 + + + + + +-6847282503429479638->-8098914186541078765 + + + + + +-6847283764628808559 + +(10, 21) + + + +7641791913198298039->-6847283764628808559 + + + + + +-6847283764628808559->-8098914186541078765 + + + + + +-6847285025832797676 + +(11, 21) + + + +1039757474567708130->-6847285025832797676 + + + + + +-6847285025832797676->-8098914186541078765 + + + + + +-6847286287032126597 + +(12, 21) + + + +-5525301545795319775->-6847286287032126597 + + + + + +-6847286287032126597->-8098914186541078765 + + + + + +-6847287548226795322 + +(13, 21) + + + +467012126175878164->-6847287548226795322 + + + + + +-6847287548226795322->-8098914186541078765 + + + + + +-6847288809426124243 + +(14, 21) + + + +1152890416563293059->-6847288809426124243 + + + + + +-6847288809426124243->-8098914186541078765 + + + + + +-6847290070611472576 + +(15, 21) + + + +-971366931658466018->-6847290070611472576 + + + + + +-6847290070611472576->-8098914186541078765 + + + + + +-6847271152621538761 + +(16, 21) + + + +4947953008258536205->-6847271152621538761 + + + + + +-6847271152621538761->-8098914186541078765 + + + + + +-6847272413816207486 + +(17, 21) + + + +6563603015227645232->-6847272413816207486 + + + + + +-6847272413816207486->-8098914186541078765 + + + + + +-6847273675015536407 + +(18, 21) + + + +-6237298120395126833->-6847273675015536407 + + + + + +-6847273675015536407->-8098914186541078765 + + + + + +-6847274936219525524 + +(19, 21) + + + +6373011764919533146->-6847274936219525524 + + + + + +-6847274936219525524->-8098914186541078765 + + + + + +-6847276197418854445 + +(20, 21) + + + +-768673660242799815->-6847276197418854445 + + + + + +-6847276197418854445->-8098914186541078765 + + + + + +-6847277458613523170 + +(21, 21) + + + +-2824362321194014932->-6847277458613523170 + + + + + +-6847277458613523170->-8098914186541078765 + + + + + +-6847278719812852091 + +(22, 21) + + + +-1731218926432692581->-6847278719812852091 + + + + + +-6847278719812852091->-8098914186541078765 + + + + + +-6847279981035481992 + +(23, 21) + + + +4834141449538341654->-6847279981035481992 + + + + + +-6847279981035481992->-8098914186541078765 + + + + + +-6847261063045548177 + +(24, 21) + + + +-1879112792699207419->-6847261063045548177 + + + + + +-6847261063045548177->-8098914186541078765 + + + + + +-6847262324240216902 + +(25, 21) + + + +-2268034439136338136->-6847262324240216902 + + + + + +-6847262324240216902->-8098914186541078765 + + + + + +-6847263585439545823 + +(26, 21) + + + +-1163384088411986841->-6847263585439545823 + + + + + +-6847263585439545823->-8098914186541078765 + + + + + +-6847264846643534940 + +(27, 21) + + + +-7432380555613506734->-6847264846643534940 + + + + + +-6847264846643534940->-8098914186541078765 + + + + + +-6847266107842863861 + +(28, 21) + + + +259646999892514961->-6847266107842863861 + + + + + +-6847266107842863861->-8098914186541078765 + + + + + +-6847267369037532586 + +(29, 21) + + + +-55747436659732284->-6847267369037532586 + + + + + +-6847267369037532586->-8098914186541078765 + + + + + +-6847268630236861507 + +(30, 21) + + + +7103861554883143731->-6847268630236861507 + + + + + +-6847268630236861507->-8098914186541078765 + + + + + +-6847269891422209840 + +(31, 21) + + + +5119391463924352654->-6847269891422209840 + + + + + +-6847269891422209840->-8098914186541078765 + + + + + +-6847250973506839161 + +(32, 21) + + + +1456041037964604477->-6847250973506839161 + + + + + +-6847250973506839161->-8098914186541078765 + + + + + +-6847252234701507886 + +(33, 21) + + + +-534437884153723040->-6847252234701507886 + + + + + +-6847252234701507886->-8098914186541078765 + + + + + +-6847253495900836807 + +(34, 21) + + + +4913406049573445695->-6847253495900836807 + + + + + +-6847253495900836807->-8098914186541078765 + + + + + +-6847254757104825924 + +(35, 21) + + + +7914639025673741002->-6847254757104825924 + + + + + +-6847254757104825924->-8098914186541078765 + + + + + +-6847256018304154845 + +(36, 21) + + + +6145630213302817897->-6847256018304154845 + + + + + +-6847256018304154845->-8098914186541078765 + + + + + +-6847257279498823570 + +(37, 21) + + + +-1699331213167282020->-6847257279498823570 + + + + + +-6847257279498823570->-8098914186541078765 + + + + + +-6847258540698152491 + +(38, 21) + + + +-207613661874215925->-6847258540698152491 + + + + + +-6847258540698152491->-8098914186541078765 + + + + + +-6847259801846219256 + +(39, 21) + + + +-1146986253355266170->-6847259801846219256 + + + + + +-6847259801846219256->-8098914186541078765 + + + + + +-6847240883930848577 + +(40, 21) + + + +6844298409384264245->-6847240883930848577 + + + + + +-6847240883930848577->-8098914186541078765 + + + + + +-6847242145125517302 + +(41, 21) + + + +-2596952707011531688->-6847242145125517302 + + + + + +-6847242145125517302->-8098914186541078765 + + + + + +-6847243406324846223 + +(42, 21) + + + +-9130766698181113193->-6847243406324846223 + + + + + +-6847243406324846223->-8098914186541078765 + + + + + +-6847244667528835340 + +(43, 21) + + + +1351867867314164930->-6847244667528835340 + + + + + +-6847244667528835340->-8098914186541078765 + + + + + +-6847245928728164261 + +(44, 21) + + + +4710342394373876225->-6847245928728164261 + + + + + +-6847245928728164261->-8098914186541078765 + + + + + +-6847247189922832986 + +(45, 21) + + + +-8821277249775408652->-6847247189922832986 + + + + + +-6847247189922832986->-8098914186541078765 + + + + + +-6847248451122161907 + +(46, 21) + + + +8479153742847714659->-6847248451122161907 + + + + + +-6847248451122161907->-8098914186541078765 + + + + + +-6847249712307510240 + +(47, 21) + + + +-26802553559900418->-6847249712307510240 + + + + + +-6847249712307510240->-8098914186541078765 + + + + + +-6847230794317576425 + +(48, 21) + + + +-2768630198877085459->-6847230794317576425 + + + + + +-6847230794317576425->-8098914186541078765 + + + + + +-6847291331807306350 + +(0, 22) + + + +5798739089968052768->-6847291331807306350 + + + + + +-8114127481031040290 + +simple-shuffle-combine + + + +-6847291331807306350->-8114127481031040290 + + + + + +-6847292593008965369 + +(1, 22) + + + +-4231490635433749315->-6847292593008965369 + + + + + +-6847292593008965369->-8114127481031040290 + + + + + +-6847293854205964192 + +(2, 22) + + + +1552128932466378046->-6847293854205964192 + + + + + +-6847293854205964192->-8114127481031040290 + + + + + +-6847295115407623211 + +(3, 22) + + + +1487887347367393803->-6847295115407623211 + + + + + +-6847295115407623211->-8114127481031040290 + + + + + +-6847296376604622034 + +(4, 22) + + + +694369751230477148->-6847296376604622034 + + + + + +-6847296376604622034->-8114127481031040290 + + + + + +-6847297637806281053 + +(5, 22) + + + +7150787558806925545->-6847297637806281053 + + + + + +-6847297637806281053->-8114127481031040290 + + + + + +-6847298899003279876 + +(6, 22) + + + +-4303280582103022326->-6847298899003279876 + + + + + +-6847298899003279876->-8114127481031040290 + + + + + +-6847300160149016543 + +(7, 22) + + + +1189617517990062695->-6847300160149016543 + + + + + +-6847300160149016543->-8114127481031040290 + + + + + +-6847281242231315766 + +(8, 22) + + + +624997638727171096->-6847281242231315766 + + + + + +-6847281242231315766->-8114127481031040290 + + + + + +-6847282503432974785 + +(9, 22) + + + +-1050920043447597387->-6847282503432974785 + + + + + +-6847282503432974785->-8114127481031040290 + + + + + +-6847283764629973608 + +(10, 22) + + + +2950208301790303798->-6847283764629973608 + + + + + +-6847283764629973608->-8114127481031040290 + + + + + +-6847285025831632627 + +(11, 22) + + + +-526846616418495645->-6847285025831632627 + + + + + +-6847285025831632627->-8114127481031040290 + + + + + +-6847286287028631450 + +(12, 22) + + + +7193080811465441716->-6847286287028631450 + + + + + +-6847286287028631450->-8114127481031040290 + + + + + +-6847287548230290469 + +(13, 22) + + + +-4773293742834941311->-6847287548230290469 + + + + + +-6847287548230290469->-8114127481031040290 + + + + + +-6847288809427289292 + +(14, 22) + + + +888375099788549634->-6847288809427289292 + + + + + +-6847288809427289292->-8114127481031040290 + + + + + +-6847290070610307527 + +(15, 22) + + + +-1879378071189612993->-6847290070610307527 + + + + + +-6847290070610307527->-8114127481031040290 + + + + + +-6847271152618043614 + +(16, 22) + + + +1412075056170588880->-6847271152618043614 + + + + + +-6847271152618043614->-8114127481031040290 + + + + + +-6847272413819702633 + +(17, 22) + + + +-7854100422467840659->-6847272413819702633 + + + + + +-6847272413819702633->-8114127481031040290 + + + + + +-6847273675016701456 + +(18, 22) + + + +-3093414328619320658->-6847273675016701456 + + + + + +-6847273675016701456->-8114127481031040290 + + + + + +-6847274936218360475 + +(19, 22) + + + +-6456753713237585029->-6847274936218360475 + + + + + +-6847274936218360475->-8114127481031040290 + + + + + +-6847276197415359298 + +(20, 22) + + + +2441181481422867660->-6847276197415359298 + + + + + +-6847276197415359298->-8114127481031040290 + + + + + +-6847277458617018317 + +(21, 22) + + + +567796597906801049->-6847277458617018317 + + + + + +-6847277458617018317->-8114127481031040290 + + + + + +-6847278719814017140 + +(22, 22) + + + +-766566939613686406->-6847278719814017140 + + + + + +-6847278719814017140->-8114127481031040290 + + + + + +-6847279981034316943 + +(23, 22) + + + +-4982041670001794921->-6847279981034316943 + + + + + +-6847279981034316943->-8114127481031040290 + + + + + +-6847261063042053030 + +(24, 22) + + + +-5924080906071941944->-6847261063042053030 + + + + + +-6847261063042053030->-8114127481031040290 + + + + + +-6847262324243712049 + +(25, 22) + + + +1316580840572601189->-6847262324243712049 + + + + + +-6847262324243712049->-8114127481031040290 + + + + + +-6847263585440710872 + +(26, 22) + + + +-1846674863145291866->-6847263585440710872 + + + + + +-6847263585440710872->-8114127481031040290 + + + + + +-6847264846642369891 + +(27, 22) + + + +-8973479904705891309->-6847264846642369891 + + + + + +-6847264846642369891->-8114127481031040290 + + + + + +-6847266107839368714 + +(28, 22) + + + +7437115520122686052->-6847266107839368714 + + + + + +-6847266107839368714->-8114127481031040290 + + + + + +-6847267369041027733 + +(29, 22) + + + +5642550134643833841->-6847267369041027733 + + + + + +-6847267369041027733->-8114127481031040290 + + + + + +-6847268630238026556 + +(30, 22) + + + +-336414278698477710->-6847268630238026556 + + + + + +-6847268630238026556->-8114127481031040290 + + + + + +-6847269891421044791 + +(31, 22) + + + +4071588535490063279->-6847269891421044791 + + + + + +-6847269891421044791->-8114127481031040290 + + + + + +-6847250973503344014 + +(32, 22) + + + +-3678441076569365248->-6847250973503344014 + + + + + +-6847250973503344014->-8114127481031040290 + + + + + +-6847252234705003033 + +(33, 22) + + + +4651466916419881885->-6847252234705003033 + + + + + +-6847252234705003033->-8114127481031040290 + + + + + +-6847253495902001856 + +(34, 22) + + + +-3716046574174494946->-6847253495902001856 + + + + + +-6847253495902001856->-8114127481031040290 + + + + + +-6847254757103660875 + +(35, 22) + + + +6932232795258523627->-6847254757103660875 + + + + + +-6847254757103660875->-8114127481031040290 + + + + + +-6847256018300659698 + +(36, 22) + + + +2779624831732602172->-6847256018300659698 + + + + + +-6847256018300659698->-8114127481031040290 + + + + + +-6847257279502318717 + +(37, 22) + + + +2324711089811746505->-6847257279502318717 + + + + + +-6847257279502318717->-8114127481031040290 + + + + + +-6847258540699317540 + +(38, 22) + + + +296313021205833450->-6847258540699317540 + + + + + +-6847258540699317540->-8114127481031040290 + + + + + +-6847259801845054207 + +(39, 22) + + + +-3261896589259605945->-6847259801845054207 + + + + + +-6847259801845054207->-8114127481031040290 + + + + + +-6847240883927353430 + +(40, 22) + + + +-4586504018197119496->-6847240883927353430 + + + + + +-6847240883927353430->-8114127481031040290 + + + + + +-6847242145129012449 + +(41, 22) + + + +-8552078854458765163->-6847242145129012449 + + + + + +-6847242145129012449->-8114127481031040290 + + + + + +-6847243406326011272 + +(42, 22) + + + +775284604603602198->-6847243406326011272 + + + + + +-6847243406326011272->-8114127481031040290 + + + + + +-6847244667527670291 + +(43, 22) + + + +8631376713219557955->-6847244667527670291 + + + + + +-6847244667527670291->-8114127481031040290 + + + + + +-6847245928724669114 + +(44, 22) + + + +-1074558119228731500->-6847245928724669114 + + + + + +-6847245928724669114->-8114127481031040290 + + + + + +-6847247189926328133 + +(45, 22) + + + +-4829218144180173727->-6847247189926328133 + + + + + +-6847247189926328133->-8114127481031040290 + + + + + +-6847248451123326956 + +(46, 22) + + + +-6866207458507005982->-6847248451123326956 + + + + + +-6847248451123326956->-8114127481031040290 + + + + + +-6847249712306345191 + +(47, 22) + + + +-2757046884319693793->-6847249712306345191 + + + + + +-6847249712306345191->-8114127481031040290 + + + + + +-6847230794314081278 + +(48, 22) + + + +-6757349109966991184->-6847230794314081278 + + + + + +-6847230794314081278->-8114127481031040290 + + + + + +-6847291331808471399 + +(0, 23) + + + +7212286209266448543->-6847291331808471399 + + + + + +1915418191650649001 + +simple-shuffle-combine + + + +-6847291331808471399->1915418191650649001 + + + + + +-6847292593007800320 + +(1, 23) + + + +-6549489177168646690->-6847292593007800320 + + + + + +-6847292593007800320->1915418191650649001 + + + + + +-6847293854207129241 + +(2, 23) + + + +3799192399535675421->-6847293854207129241 + + + + + +-6847293854207129241->1915418191650649001 + + + + + +-6847295115406458162 + +(3, 23) + + + +1752407194656485628->-6847295115406458162 + + + + + +-6847295115406458162->1915418191650649001 + + + + + +-6847296376605787083 + +(4, 23) + + + +1200219479784636523->-6847296376605787083 + + + + + +-6847296376605787083->1915418191650649001 + + + + + +-6847297637805116004 + +(5, 23) + + + +5215974249926099370->-6847297637805116004 + + + + + +-6847297637805116004->1915418191650649001 + + + + + +-6847298899004444925 + +(6, 23) + + + +6241526678049256265->-6847298899004444925 + + + + + +-6847298899004444925->1915418191650649001 + + + + + +-6847300160147851494 + +(7, 23) + + + +1899615472660797320->-6847300160147851494 + + + + + +-6847300160147851494->1915418191650649001 + + + + + +-6847281242232480815 + +(8, 23) + + + +2850871046876046071->-6847281242232480815 + + + + + +-6847281242232480815->1915418191650649001 + + + + + +-6847282503431809736 + +(9, 23) + + + +-1957845225491153962->-6847282503431809736 + + + + + +-6847282503431809736->1915418191650649001 + + + + + +-6847283764631138657 + +(10, 23) + + + +-5445140947235528427->-6847283764631138657 + + + + + +-6847283764631138657->1915418191650649001 + + + + + +-6847285025830467578 + +(11, 23) + + + +8173541034346624596->-6847285025830467578 + + + + + +-6847285025830467578->1915418191650649001 + + + + + +-6847286287029796499 + +(12, 23) + + + +1456537633514785475->-6847286287029796499 + + + + + +-6847286287029796499->1915418191650649001 + + + + + +-6847287548229125420 + +(13, 23) + + + +3007127152366814114->-6847287548229125420 + + + + + +-6847287548229125420->1915418191650649001 + + + + + +-6847288809428454341 + +(14, 23) + + + +-5903375708756309791->-6847288809428454341 + + + + + +-6847288809428454341->1915418191650649001 + + + + + +-6847290070609142478 + +(15, 23) + + + +-3986553634195349568->-6847290070609142478 + + + + + +-6847290070609142478->1915418191650649001 + + + + + +-6847271152619208663 + +(16, 23) + + + +-6491105168949583345->-6847271152619208663 + + + + + +-6847271152619208663->1915418191650649001 + + + + + +-6847272413818537584 + +(17, 23) + + + +-342871098721008818->-6847272413818537584 + + + + + +-6847272413818537584->1915418191650649001 + + + + + +-6847273675017866505 + +(18, 23) + + + +-859769094525093683->-6847273675017866505 + + + + + +-6847273675017866505->1915418191650649001 + + + + + +-6847274936217195426 + +(19, 23) + + + +2091689592548223596->-6847274936217195426 + + + + + +-6847274936217195426->1915418191650649001 + + + + + +-6847276197416524347 + +(20, 23) + + + +-6036326804014115365->-6847276197416524347 + + + + + +-6847276197416524347->1915418191650649001 + + + + + +-6847277458615853268 + +(21, 23) + + + +1423159941966510874->-6847277458615853268 + + + + + +-6847277458615853268->1915418191650649001 + + + + + +-6847278719815182189 + +(22, 23) + + + +-341424509669940231->-6847278719815182189 + + + + + +-6847278719815182189->1915418191650649001 + + + + + +-6847279981033151894 + +(23, 23) + + + +1798209393923621304->-6847279981033151894 + + + + + +-6847279981033151894->1915418191650649001 + + + + + +-6847261063043218079 + +(24, 23) + + + +-5628780061627079769->-6847261063043218079 + + + + + +-6847261063043218079->1915418191650649001 + + + + + +-6847262324242547000 + +(25, 23) + + + +272076044346687814->-6847262324242547000 + + + + + +-6847262324242547000->1915418191650649001 + + + + + +-6847263585441875921 + +(26, 23) + + + +-944179717246316091->-6847263585441875921 + + + + + +-6847263585441875921->1915418191650649001 + + + + + +-6847264846641204842 + +(27, 23) + + + +8421030140207452932->-6847264846641204842 + + + + + +-6847264846641204842->1915418191650649001 + + + + + +-6847266107840533763 + +(28, 23) + + + +6660611395198675827->-6847266107840533763 + + + + + +-6847266107840533763->1915418191650649001 + + + + + +-6847267369039862684 + +(29, 23) + + + +2463198802763963666->-6847267369039862684 + + + + + +-6847267369039862684->1915418191650649001 + + + + + +-6847268630239191605 + +(30, 23) + + + +-7210338045358997935->-6847268630239191605 + + + + + +-6847268630239191605->1915418191650649001 + + + + + +-6847269891419879742 + +(31, 23) + + + +-7555982554687872912->-6847269891419879742 + + + + + +-6847269891419879742->1915418191650649001 + + + + + +-6847250973504509063 + +(32, 23) + + + +-3886288245780332673->-6847250973504509063 + + + + + +-6847250973504509063->1915418191650649001 + + + + + +-6847252234703837984 + +(33, 23) + + + +3615821094161925310->-6847252234703837984 + + + + + +-6847252234703837984->1915418191650649001 + + + + + +-6847253495903166905 + +(34, 23) + + + +6739861364967336445->-6847253495903166905 + + + + + +-6847253495903166905->1915418191650649001 + + + + + +-6847254757102495826 + +(35, 23) + + + +3759914786213740252->-6847254757102495826 + + + + + +-6847254757102495826->1915418191650649001 + + + + + +-6847256018301824747 + +(36, 23) + + + +4708742265902729803->-6847256018301824747 + + + + + +-6847256018301824747->1915418191650649001 + + + + + +-6847257279501153668 + +(37, 23) + + + +767553786986171530->-6847257279501153668 + + + + + +-6847257279501153668->1915418191650649001 + + + + + +-6847258540700482589 + +(38, 23) + + + +2385718588681501225->-6847258540700482589 + + + + + +-6847258540700482589->1915418191650649001 + + + + + +-6847259801843889158 + +(39, 23) + + + +-4108119666275828120->-6847259801843889158 + + + + + +-6847259801843889158->1915418191650649001 + + + + + +-6847240883928518479 + +(40, 23) + + + +-2422806564999617321->-6847240883928518479 + + + + + +-6847240883928518479->1915418191650649001 + + + + + +-6847242145127847400 + +(41, 23) + + + +70770096275611062->-6847242145127847400 + + + + + +-6847242145127847400->1915418191650649001 + + + + + +-6847243406327176321 + +(42, 23) + + + +-4906100434311288843->-6847243406327176321 + + + + + +-6847243406327176321->1915418191650649001 + + + + + +-6847244667526505242 + +(43, 23) + + + +8206131122471120180->-6847244667526505242 + + + + + +-6847244667526505242->1915418191650649001 + + + + + +-6847245928725834163 + +(44, 23) + + + +-814655595134424925->-6847245928725834163 + + + + + +-6847245928725834163->1915418191650649001 + + + + + +-6847247189925163084 + +(45, 23) + + + +-6354393384697048702->-6847247189925163084 + + + + + +-6847247189925163084->1915418191650649001 + + + + + +-6847248451124492005 + +(46, 23) + + + +-5738475575268415807->-6847248451124492005 + + + + + +-6847248451124492005->1915418191650649001 + + + + + +-6847249712305180142 + +(47, 23) + + + +-2620134805293404768->-6847249712305180142 + + + + + +-6847249712305180142->1915418191650649001 + + + + + +-6847230794315246327 + +(48, 23) + + + +4179256065765470191->-6847230794315246327 + + + + + +-6847230794315246327->1915418191650649001 + + + + + +-6847291331823617036 + +(0, 24) + + + +-1663520206827355198->-6847291331823617036 + + + + + +-4973298990768712940 + +simple-shuffle-combine + + + +-6847291331823617036->-4973298990768712940 + + + + + +-6847292593020615859 + +(1, 24) + + + +8773609046835760035->-6847292593020615859 + + + + + +-6847292593020615859->-4973298990768712940 + + + + + +-6847293854217614682 + +(2, 24) + + + +-2048179483005988620->-6847293854217614682 + + + + + +-6847293854217614682->-4973298990768712940 + + + + + +-6847295115414613505 + +(3, 24) + + + +-8263668584995597963->-6847295115414613505 + + + + + +-6847295115414613505->-4973298990768712940 + + + + + +-6847296376611612328 + +(4, 24) + + + +-9082321085600011018->-6847296376611612328 + + + + + +-6847296376611612328->-4973298990768712940 + + + + + +-6847297637808611151 + +(5, 24) + + + +-7516804374939175721->-6847297637808611151 + + + + + +-6847297637808611151->-4973298990768712940 + + + + + +-6847298899005609974 + +(6, 24) + + + +7218399053394967640->-6847298899005609974 + + + + + +-6847298899005609974->-4973298990768712940 + + + + + +-6847300160146686445 + +(7, 24) + + + +1000458774366247545->-6847300160146686445 + + + + + +-6847300160146686445->-4973298990768712940 + + + + + +-6847281242228985668 + +(8, 24) + + + +8743900272582271946->-6847281242228985668 + + + + + +-6847281242228985668->-4973298990768712940 + + + + + +-6847282503425984491 + +(9, 24) + + + +1006443045261147979->-6847282503425984491 + + + + + +-6847282503425984491->-4973298990768712940 + + + + + +-6847283764622983314 + +(10, 24) + + + +-6880333718694692452->-6847283764622983314 + + + + + +-6847283764622983314->-4973298990768712940 + + + + + +-6847285025819982137 + +(11, 24) + + + +5532796804919332221->-6847285025819982137 + + + + + +-6847285025819982137->-4973298990768712940 + + + + + +-6847286287016980960 + +(12, 24) + + + +3430295500231488766->-6847286287016980960 + + + + + +-6847286287016980960->-4973298990768712940 + + + + + +-6847287548213979783 + +(13, 24) + + + +2140391653956662655->-6847287548213979783 + + + + + +-6847287548213979783->-4973298990768712940 + + + + + +-6847288809410978606 + +(14, 24) + + + +1172979360121986400->-6847288809410978606 + + + + + +-6847288809410978606->-4973298990768712940 + + + + + +-6847290070626618213 + +(15, 24) + + + +9136614920439784257->-6847290070626618213 + + + + + +-6847290070626618213->-4973298990768712940 + + + + + +-6847271152634354300 + +(16, 24) + + + +2959662237077566514->-6847271152634354300 + + + + + +-6847271152634354300->-4973298990768712940 + + + + + +-6847272413831353123 + +(17, 24) + + + +-5280607259195937709->-6847272413831353123 + + + + + +-6847272413831353123->-4973298990768712940 + + + + + +-6847273675028351946 + +(18, 24) + + + +735041124370256292->-6847273675028351946 + + + + + +-6847273675028351946->-4973298990768712940 + + + + + +-6847274936225350769 + +(19, 24) + + + +3345275684338122021->-6847274936225350769 + + + + + +-6847274936225350769->-4973298990768712940 + + + + + +-6847276197422349592 + +(20, 24) + + + +-6175283792830314906->-6847276197422349592 + + + + + +-6847276197422349592->-4973298990768712940 + + + + + +-6847277458619348415 + +(21, 24) + + + +3034680462950520199->-6847277458619348415 + + + + + +-6847277458619348415->-4973298990768712940 + + + + + +-6847278719816347238 + +(22, 24) + + + +-7699655287504900856->-6847278719816347238 + + + + + +-6847278719816347238->-4973298990768712940 + + + + + +-6847279981031986845 + +(23, 24) + + + +754835437555138729->-6847279981031986845 + + + + + +-6847279981031986845->-4973298990768712940 + + + + + +-6847261063039722932 + +(24, 24) + + + +-8517414290680849094->-6847261063039722932 + + + + + +-6847261063039722932->-4973298990768712940 + + + + + +-6847262324236721755 + +(25, 24) + + + +1407949456721687739->-6847262324236721755 + + + + + +-6847262324236721755->-4973298990768712940 + + + + + +-6847263585433720578 + +(26, 24) + + + +7863236444028396300->-6847263585433720578 + + + + + +-6847263585433720578->-4973298990768712940 + + + + + +-6847264846630719401 + +(27, 24) + + + +-5369699232878898643->-6847264846630719401 + + + + + +-6847264846630719401->-4973298990768712940 + + + + + +-6847266107827718224 + +(28, 24) + + + +-7475563045267895698->-6847266107827718224 + + + + + +-6847266107827718224->-4973298990768712940 + + + + + +-6847267369024717047 + +(29, 24) + + + +-4627265585618549265->-6847267369024717047 + + + + + +-6847267369024717047->-4973298990768712940 + + + + + +-6847268630221715870 + +(30, 24) + + + +-2396449631314895344->-6847268630221715870 + + + + + +-6847268630221715870->-4973298990768712940 + + + + + +-6847269891437355477 + +(31, 24) + + + +4937201793546689713->-6847269891437355477 + + + + + +-6847269891437355477->-4973298990768712940 + + + + + +-6847250973519654700 + +(32, 24) + + + +-1728304742676626014->-6847250973519654700 + + + + + +-6847250973519654700->-4973298990768712940 + + + + + +-6847252234716653523 + +(33, 24) + + + +-712197553789702781->-6847252234716653523 + + + + + +-6847252234716653523->-4973298990768712940 + + + + + +-6847253495913652346 + +(34, 24) + + + +1717133504382587604->-6847253495913652346 + + + + + +-6847253495913652346->-4973298990768712940 + + + + + +-6847254757110651169 + +(35, 24) + + + +-6239515116878932139->-6847254757110651169 + + + + + +-6847254757110651169->-4973298990768712940 + + + + + +-6847256018307649992 + +(36, 24) + + + +1832782470767640278->-6847256018307649992 + + + + + +-6847256018307649992->-4973298990768712940 + + + + + +-6847257279504648815 + +(37, 24) + + + +6496178653449948855->-6847257279504648815 + + + + + +-6847257279504648815->-4973298990768712940 + + + + + +-6847258540701647638 + +(38, 24) + + + +-5592076947777263816->-6847258540701647638 + + + + + +-6847258540701647638->-4973298990768712940 + + + + + +-6847259801842724109 + +(39, 24) + + + +-5235847009058160295->-6847259801842724109 + + + + + +-6847259801842724109->-4973298990768712940 + + + + + +-6847240883925023332 + +(40, 24) + + + +2547315866271084970->-6847240883925023332 + + + + + +-6847240883925023332->-4973298990768712940 + + + + + +-6847242145122022155 + +(41, 24) + + + +-4300885774567123413->-6847242145122022155 + + + + + +-6847242145122022155->-4973298990768712940 + + + + + +-6847243406319020978 + +(42, 24) + + + +2199316226298567548->-6847243406319020978 + + + + + +-6847243406319020978->-4973298990768712940 + + + + + +-6847244667516019801 + +(43, 24) + + + +-5583222122110866595->-6847244667516019801 + + + + + +-6847244667516019801->-4973298990768712940 + + + + + +-6847245928713018624 + +(44, 24) + + + +-7676869065391836450->-6847245928713018624 + + + + + +-6847245928713018624->-4973298990768712940 + + + + + +-6847247189910017447 + +(45, 24) + + + +-5625004660244326561->-6847247189910017447 + + + + + +-6847247189910017447->-4973298990768712940 + + + + + +-6847248451107016270 + +(46, 24) + + + +-8302828196307841216->-6847248451107016270 + + + + + +-6847248451107016270->-4973298990768712940 + + + + + +-6847249712322655877 + +(47, 24) + + + +308684571650206241->-6847249712322655877 + + + + + +-6847249712322655877->-4973298990768712940 + + + + + +-6847230794330391964 + +(48, 24) + + + +-7042257413225562350->-6847230794330391964 + + + + + +-6847230794330391964->-4973298990768712940 + + + + + +-6847291331824782085 + +(0, 25) + + + +-9174731103954879839->-6847291331824782085 + + + + + +-3802725357299598065 + +simple-shuffle-combine + + + +-6847291331824782085->-3802725357299598065 + + + + + +-6847292593019450810 + +(1, 25) + + + +8442008876282599060->-6847292593019450810 + + + + + +-6847292593019450810->-3802725357299598065 + + + + + +-6847293854218779731 + +(2, 25) + + + +-361754552017269245->-6847293854218779731 + + + + + +-6847293854218779731->-3802725357299598065 + + + + + +-6847295115413448456 + +(3, 25) + + + +8768407791115709078->-6847295115413448456 + + + + + +-6847295115413448456->-3802725357299598065 + + + + + +-6847296376612777377 + +(4, 25) + + + +-8038925783830374443->-6847296376612777377 + + + + + +-6847296376612777377->-3802725357299598065 + + + + + +-6847297637807446102 + +(5, 25) + + + +8195266512355184120->-6847297637807446102 + + + + + +-6847297637807446102->-3802725357299598065 + + + + + +-6847298899006775023 + +(6, 25) + + + +8811184276335087415->-6847298899006775023 + + + + + +-6847298899006775023->-3802725357299598065 + + + + + +-6847300160145521396 + +(7, 25) + + + +-2197461496226689030->-6847300160145521396 + + + + + +-6847300160145521396->-3802725357299598065 + + + + + +-6847281242230150717 + +(8, 25) + + + +-8575075640969064695->-6847281242230150717 + + + + + +-6847281242230150717->-3802725357299598065 + + + + + +-6847282503424819442 + +(9, 25) + + + +7793764927223571004->-6847282503424819442 + + + + + +-6847282503424819442->-3802725357299598065 + + + + + +-6847283764624148363 + +(10, 25) + + + +3789311093500666539->-6847283764624148363 + + + + + +-6847283764624148363->-3802725357299598065 + + + + + +-6847285025818817088 + +(11, 25) + + + +4489404900719359646->-6847285025818817088 + + + + + +-6847285025818817088->-3802725357299598065 + + + + + +-6847286287018146009 + +(12, 25) + + + +4403823161694539741->-6847286287018146009 + + + + + +-6847286287018146009->-3802725357299598065 + + + + + +-6847287548212814734 + +(13, 25) + + + +1095912667101274880->-6847287548212814734 + + + + + +-6847287548212814734->-3802725357299598065 + + + + + +-6847288809412143655 + +(14, 25) + + + +4419821694119671775->-6847288809412143655 + + + + + +-6847288809412143655->-3802725357299598065 + + + + + +-6847290070625453164 + +(15, 25) + + + +7546500573262659682->-6847290070625453164 + + + + + +-6847290070625453164->-3802725357299598065 + + + + + +-6847271152635519349 + +(16, 25) + + + +-5598784686580662511->-6847271152635519349 + + + + + +-6847271152635519349->-3802725357299598065 + + + + + +-6847272413830188074 + +(17, 25) + + + +-6407230737691217084->-6847272413830188074 + + + + + +-6847272413830188074->-3802725357299598065 + + + + + +-6847273675029516995 + +(18, 25) + + + +-5008401621895459149->-6847273675029516995 + + + + + +-6847273675029516995->-3802725357299598065 + + + + + +-6847274936224185720 + +(19, 25) + + + +-7147208146494776570->-6847274936224185720 + + + + + +-6847274936224185720->-3802725357299598065 + + + + + +-6847276197423514641 + +(20, 25) + + + +-6439785698869668731->-6847276197423514641 + + + + + +-6847276197423514641->-3802725357299598065 + + + + + +-6847277458618183366 + +(21, 25) + + + +-6719462818331743576->-6847277458618183366 + + + + + +-6847277458618183366->-3802725357299598065 + + + + + +-6847278719817512287 + +(22, 25) + + + +5018333618329145319->-6847278719817512287 + + + + + +-6847278719817512287->-3802725357299598065 + + + + + +-6847279981030821796 + +(23, 25) + + + +8330505878823518570->-6847279981030821796 + + + + + +-6847279981030821796->-3802725357299598065 + + + + + +-6847261063040887981 + +(24, 25) + + + +-8663179631476456519->-6847261063040887981 + + + + + +-6847261063040887981->-3802725357299598065 + + + + + +-6847262324235556706 + +(25, 25) + + + +8918088318100772780->-6847262324235556706 + + + + + +-6847262324235556706->-3802725357299598065 + + + + + +-6847263585434885627 + +(26, 25) + + + +-6205851716954074341->-6847263585434885627 + + + + + +-6847263585434885627->-3802725357299598065 + + + + + +-6847264846629554352 + +(27, 25) + + + +-6858996734189228018->-6847264846629554352 + + + + + +-6847264846629554352->-3802725357299598065 + + + + + +-6847266107828883273 + +(28, 25) + + + +-6427737742687294323->-6847266107828883273 + + + + + +-6847266107828883273->-3802725357299598065 + + + + + +-6847267369023551998 + +(29, 25) + + + +2629966072257693360->-6847267369023551998 + + + + + +-6847267369023551998->-3802725357299598065 + + + + + +-6847268630222880919 + +(30, 25) + + + +-828737145648533169->-6847268630222880919 + + + + + +-6847268630222880919->-3802725357299598065 + + + + + +-6847269891436190428 + +(31, 25) + + + +-4947639894027167278->-6847269891436190428 + + + + + +-6847269891436190428->-3802725357299598065 + + + + + +-6847250973520819749 + +(32, 25) + + + +-674962311354119039->-6847250973520819749 + + + + + +-6847250973520819749->-3802725357299598065 + + + + + +-6847252234715488474 + +(33, 25) + + + +-2678440043729743756->-6847252234715488474 + + + + + +-6847252234715488474->-3802725357299598065 + + + + + +-6847253495914817395 + +(34, 25) + + + +3206041209588625379->-6847253495914817395 + + + + + +-6847253495914817395->-3802725357299598065 + + + + + +-6847254757109486120 + +(35, 25) + + + +-7211975355068024714->-6847254757109486120 + + + + + +-6847254757109486120->-3802725357299598065 + + + + + +-6847256018308815041 + +(36, 25) + + + +2877282724268323253->-6847256018308815041 + + + + + +-6847256018308815041->-3802725357299598065 + + + + + +-6847257279503483766 + +(37, 25) + + + +-5234313519220125736->-6847257279503483766 + + + + + +-6847257279503483766->-3802725357299598065 + + + + + +-6847258540702812687 + +(38, 25) + + + +4377155478415140375->-6847258540702812687 + + + + + +-6847258540702812687->-3802725357299598065 + + + + + +-6847259801841559060 + +(39, 25) + + + +-6701451917992012070->-6847259801841559060 + + + + + +-6847259801841559060->-3802725357299598065 + + + + + +-6847240883926188381 + +(40, 25) + + + +2281687721763725545->-6847240883926188381 + + + + + +-6847240883926188381->-3802725357299598065 + + + + + +-6847242145120857106 + +(41, 25) + + + +-7110957025498933988->-6847242145120857106 + + + + + +-6847242145120857106->-3802725357299598065 + + + + + +-6847243406320186027 + +(42, 25) + + + +3730024123555754123->-6847243406320186027 + + + + + +-6847243406320186027->-3802725357299598065 + + + + + +-6847244667514854752 + +(43, 25) + + + +-6618863447161603970->-6847244667514854752 + + + + + +-6847244667514854752->-3802725357299598065 + + + + + +-6847245928714183673 + +(44, 25) + + + +-6713304088032235075->-6847245928714183673 + + + + + +-6847245928714183673->-3802725357299598065 + + + + + +-6847247189908852398 + +(45, 25) + + + +-8379935720023823136->-6847247189908852398 + + + + + +-6847247189908852398->-3802725357299598065 + + + + + +-6847248451108181319 + +(46, 25) + + + +-6679587751915480641->-6847248451108181319 + + + + + +-6847248451108181319->-3802725357299598065 + + + + + +-6847249712321490828 + +(47, 25) + + + +-599331097234822334->-6847249712321490828 + + + + + +-6847249712321490828->-3802725357299598065 + + + + + +-6847230794331557013 + +(48, 25) + + + +-6068710119019829775->-6847230794331557013 + + + + + +-6847230794331557013->-3802725357299598065 + + + + + +-6847291331821286938 + +(0, 26) + + + +5247423636651724852->-6847291331821286938 + + + + + +6222690932248597626 + +simple-shuffle-combine + + + +-6847291331821286938->6222690932248597626 + + + + + +-6847292593022945957 + +(1, 26) + + + +-8304391524974527231->-6847292593022945957 + + + + + +-6847292593022945957->6222690932248597626 + + + + + +-6847293854219944780 + +(2, 26) + + + +562870606765491330->-6847293854219944780 + + + + + +-6847293854219944780->6222690932248597626 + + + + + +-6847295115412283407 + +(3, 26) + + + +5962798397107553303->-6847295115412283407 + + + + + +-6847295115412283407->6222690932248597626 + + + + + +-6847296376609282230 + +(4, 26) + + + +4017953867954020248->-6847296376609282230 + + + + + +-6847296376609282230->6222690932248597626 + + + + + +-6847297637810941249 + +(5, 26) + + + +-5067652299550272971->-6847297637810941249 + + + + + +-6847297637810941249->6222690932248597626 + + + + + +-6847298899007940072 + +(6, 26) + + + +1291081524850421174->-6847298899007940072 + + + + + +-6847298899007940072->6222690932248597626 + + + + + +-6847300160144356347 + +(7, 26) + + + +-3241944988014194405->-6847300160144356347 + + + + + +-6847300160144356347->6222690932248597626 + + + + + +-6847281242226655570 + +(8, 26) + + + +3388576755599659996->-6847281242226655570 + + + + + +-6847281242226655570->6222690932248597626 + + + + + +-6847282503428314589 + +(9, 26) + + + +-5451425430555060887->-6847282503428314589 + + + + + +-6847282503428314589->6222690932248597626 + + + + + +-6847283764625313412 + +(10, 26) + + + +-3720827768969603702->-6847283764625313412 + + + + + +-6847283764625313412->6222690932248597626 + + + + + +-6847285025817652039 + +(11, 26) + + + +-5962897762206347329->-6847285025817652039 + + + + + +-6847285025817652039->6222690932248597626 + + + + + +-6847286287014650862 + +(12, 26) + + + +-794393757369996384->-6847286287014650862 + + + + + +-6847286287014650862->6222690932248597626 + + + + + +-6847287548216309881 + +(13, 26) + + + +7500470524465785405->-6847287548216309881 + + + + + +-6847287548216309881->6222690932248597626 + + + + + +-6847288809413308704 + +(14, 26) + + + +5472054629946677950->-6847288809413308704 + + + + + +-6847288809413308704->6222690932248597626 + + + + + +-6847290070624288115 + +(15, 26) + + + +4874965371404084707->-6847290070624288115 + + + + + +-6847290070624288115->6222690932248597626 + + + + + +-6847271152632024202 + +(16, 26) + + + +7585068148036350180->-6847271152632024202 + + + + + +-6847271152632024202->6222690932248597626 + + + + + +-6847272413833683221 + +(17, 26) + + + +-12954114385398159->-6847272413833683221 + + + + + +-6847272413833683221->6222690932248597626 + + + + + +-6847273675030682044 + +(18, 26) + + + +4959726977664441842->-6847273675030682044 + + + + + +-6847273675030682044->6222690932248597626 + + + + + +-6847274936223020671 + +(19, 26) + + + +-8190582144969151545->-6847274936223020671 + + + + + +-6847274936223020671->6222690932248597626 + + + + + +-6847276197420019494 + +(20, 26) + + + +775747295967653960->-6847276197420019494 + + + + + +-6847276197420019494->6222690932248597626 + + + + + +-6847277458621678513 + +(21, 26) + + + +5645784816305451749->-6847277458621678513 + + + + + +-6847277458621678513->6222690932248597626 + + + + + +-6847278719818677336 + +(22, 26) + + + +4801632114246157094->-6847278719818677336 + + + + + +-6847278719818677336->6222690932248597626 + + + + + +-6847279981029656747 + +(23, 26) + + + +-3474279315751989621->-6847279981029656747 + + + + + +-6847279981029656747->6222690932248597626 + + + + + +-6847261063037392834 + +(24, 26) + + + +6857503357280599372->-6847261063037392834 + + + + + +-6847261063037392834->6222690932248597626 + + + + + +-6847262324239051853 + +(25, 26) + + + +-5464031639798759911->-6847262324239051853 + + + + + +-6847262324239051853->6222690932248597626 + + + + + +-6847263585436050676 + +(26, 26) + + + +-5241199648105653766->-6847263585436050676 + + + + + +-6847263585436050676->6222690932248597626 + + + + + +-6847264846628389303 + +(27, 26) + + + +428261545813756207->-6847264846628389303 + + + + + +-6847264846628389303->6222690932248597626 + + + + + +-6847266107825388126 + +(28, 26) + + + +-419292450384432048->-6847266107825388126 + + + + + +-6847266107825388126->6222690932248597626 + + + + + +-6847267369027047145 + +(29, 26) + + + +-4438088943976714515->-6847267369027047145 + + + + + +-6847267369027047145->6222690932248597626 + + + + + +-6847268630224045968 + +(30, 26) + + + +9072885264865458222->-6847268630224045968 + + + + + +-6847268630224045968->6222690932248597626 + + + + + +-6847269891435025379 + +(31, 26) + + + +1865375296871548563->-6847269891435025379 + + + + + +-6847269891435025379->6222690932248597626 + + + + + +-6847250973517324602 + +(32, 26) + + + +-5948559388135811564->-6847250973517324602 + + + + + +-6847250973517324602->6222690932248597626 + + + + + +-6847252234718983621 + +(33, 26) + + + +690885871977081569->-6847252234718983621 + + + + + +-6847252234718983621->6222690932248597626 + + + + + +-6847253495915982444 + +(34, 26) + + + +4253979053165625954->-6847253495915982444 + + + + + +-6847253495915982444->6222690932248597626 + + + + + +-6847254757108321071 + +(35, 26) + + + +689125325670827511->-6847254757108321071 + + + + + +-6847254757108321071->6222690932248597626 + + + + + +-6847256018305319894 + +(36, 26) + + + +8409280786164305272->-6847256018305319894 + + + + + +-6847256018305319894->6222690932248597626 + + + + + +-6847257279506978913 + +(37, 26) + + + +7880034129525085205->-6847257279506978913 + + + + + +-6847257279506978913->6222690932248597626 + + + + + +-6847258540703977736 + +(38, 26) + + + +4093817371854054550->-6847258540703977736 + + + + + +-6847258540703977736->6222690932248597626 + + + + + +-6847259801840394011 + +(39, 26) + + + +-7737075337351792645->-6847259801840394011 + + + + + +-6847259801840394011->6222690932248597626 + + + + + +-6847240883922693234 + +(40, 26) + + + +-2812478587456512580->-6847240883922693234 + + + + + +-6847240883922693234->6222690932248597626 + + + + + +-6847242145124352253 + +(41, 26) + + + +6029141879724942153->-6847242145124352253 + + + + + +-6847242145124352253->6222690932248597626 + + + + + +-6847243406321351076 + +(42, 26) + + + +4773402691324853098->-6847243406321351076 + + + + + +-6847243406321351076->6222690932248597626 + + + + + +-6847244667513689703 + +(43, 26) + + + +-5907855711508110945->-6847244667513689703 + + + + + +-6847244667513689703->6222690932248597626 + + + + + +-6847245928710688526 + +(44, 26) + + + +8176237607439769216->-6847245928710688526 + + + + + +-6847245928710688526->6222690932248597626 + + + + + +-6847247189912347545 + +(45, 26) + + + +-1492443423920592611->-6847247189912347545 + + + + + +-6847247189912347545->6222690932248597626 + + + + + +-6847248451109346368 + +(46, 26) + + + +-5635105395237709666->-6847248451109346368 + + + + + +-6847248451109346368->6222690932248597626 + + + + + +-6847249712320325779 + +(47, 26) + + + +-4981143725604578109->-6847249712320325779 + + + + + +-6847249712320325779->6222690932248597626 + + + + + +-6847230794328061866 + +(48, 26) + + + +-131318039171779900->-6847230794328061866 + + + + + +-6847230794328061866->6222690932248597626 + + + + + +-6847291331822451987 + +(0, 27) + + + +5100549947611951427->-6847291331822451987 + + + + + +8426695571241120901 + +simple-shuffle-combine + + + +-6847291331822451987->8426695571241120901 + + + + + +-6847292593021780908 + +(1, 27) + + + +-8599728700690429406->-6847292593021780908 + + + + + +-6847292593021780908->8426695571241120901 + + + + + +-6847293854221109829 + +(2, 27) + + + +2489933337640228705->-6847293854221109829 + + + + + +-6847293854221109829->8426695571241120901 + + + + + +-6847295115411118358 + +(3, 27) + + + +4998146337127177528->-6847295115411118358 + + + + + +-6847295115411118358->8426695571241120901 + + + + + +-6847296376610447279 + +(4, 27) + + + +5065757902676898423->-6847296376610447279 + + + + + +-6847296376610447279->8426695571241120901 + + + + + +-6847297637809776200 + +(5, 27) + + + +-5965723049242597546->-6847297637809776200 + + + + + +-6847297637809776200->8426695571241120901 + + + + + +-6847298899009105121 + +(6, 27) + + + +-7734715135337881451->-6847298899009105121 + + + + + +-6847298899009105121->8426695571241120901 + + + + + +-6847300160143191298 + +(7, 27) + + + +-5926207496342939380->-6847300160143191298 + + + + + +-6847300160143191298->8426695571241120901 + + + + + +-6847281242227820619 + +(8, 27) + + + +4433055698062862571->-6847281242227820619 + + + + + +-6847281242227820619->8426695571241120901 + + + + + +-6847282503427149540 + +(9, 27) + + + +-6432723295007343062->-6847282503427149540 + + + + + +-6847282503427149540->8426695571241120901 + + + + + +-6847283764626478461 + +(10, 27) + + + +-3290987550281756727->-6847283764626478461 + + + + + +-6847283764626478461->8426695571241120901 + + + + + +-6847285025816486990 + +(11, 27) + + + +1325470021017036096->-6847285025816486990 + + + + + +-6847285025816486990->8426695571241120901 + + + + + +-6847286287015815911 + +(12, 27) + + + +1369402351752040991->-6847286287015815911 + + + + + +-6847286287015815911->8426695571241120901 + + + + + +-6847287548215144832 + +(13, 27) + + + +6455970341909460830->-6847287548215144832 + + + + + +-6847287548215144832->8426695571241120901 + + + + + +-6847288809414473753 + +(14, 27) + + + +5803640844431655325->-6847288809414473753 + + + + + +-6847288809414473753->8426695571241120901 + + + + + +-6847290070623123066 + +(15, 27) + + + +5092775270219192532->-6847290070623123066 + + + + + +-6847290070623123066->8426695571241120901 + + + + + +-6847271152633189251 + +(16, 27) + + + +8566366047978131955->-6847271152633189251 + + + + + +-6847271152633189251->8426695571241120901 + + + + + +-6847272413832518172 + +(17, 27) + + + +-1047437420216500334->-6847272413832518172 + + + + + +-6847272413832518172->8426695571241120901 + + + + + +-6847273675031847093 + +(18, 27) + + + +4092187568622481617->-6847273675031847093 + + + + + +-6847273675031847093->8426695571241120901 + + + + + +-6847274936221855622 + +(19, 27) + + + +575032014101573096->-6847274936221855622 + + + + + +-6847274936221855622->8426695571241120901 + + + + + +-6847276197421184543 + +(20, 27) + + + +-7844810031478616281->-6847276197421184543 + + + + + +-6847276197421184543->8426695571241120901 + + + + + +-6847277458620513464 + +(21, 27) + + + +5836964808987263174->-6847277458620513464 + + + + + +-6847277458620513464->8426695571241120901 + + + + + +-6847278719819842385 + +(22, 27) + + + +-1963488869103635131->-6847278719819842385 + + + + + +-6847278719819842385->8426695571241120901 + + + + + +-6847279981028491698 + +(23, 27) + + + +-4447790321901080196->-6847279981028491698 + + + + + +-6847279981028491698->8426695571241120901 + + + + + +-6847261063038557883 + +(24, 27) + + + +8343053033217389147->-6847261063038557883 + + + + + +-6847261063038557883->8426695571241120901 + + + + + +-6847262324237886804 + +(25, 27) + + + +-5247330108072413286->-6847262324237886804 + + + + + +-6847262324237886804->8426695571241120901 + + + + + +-6847263585437215725 + +(26, 27) + + + +-5388069941740813191->-6847263585437215725 + + + + + +-6847263585437215725->8426695571241120901 + + + + + +-6847264846627224254 + +(27, 27) + + + +9028753949775256048->-6847264846627224254 + + + + + +-6847264846627224254->8426695571241120901 + + + + + +-6847266107826553175 + +(28, 27) + + + +-7826413541824066673->-6847266107826553175 + + + + + +-6847266107826553175->8426695571241120901 + + + + + +-6847267369025882096 + +(29, 27) + + + +-5411652998856400690->-6847267369025882096 + + + + + +-6847267369025882096->8426695571241120901 + + + + + +-6847268630225211017 + +(30, 27) + + + +-8329358555274128819->-6847268630225211017 + + + + + +-6847268630225211017->8426695571241120901 + + + + + +-6847269891433860330 + +(31, 27) + + + +813141233711667588->-6847269891433860330 + + + + + +-6847269891433860330->8426695571241120901 + + + + + +-6847250973518489651 + +(32, 27) + + + +-2705172533295136989->-6847250973518489651 + + + + + +-6847250973518489651->8426695571241120901 + + + + + +-6847252234717818572 + +(33, 27) + + + +332284831743854594->-6847252234717818572 + + + + + +-6847252234717818572->8426695571241120901 + + + + + +-6847253495917147493 + +(34, 27) + + + +5818619245760376129->-6847253495917147493 + + + + + +-6847253495917147493->8426695571241120901 + + + + + +-6847254757107156022 + +(35, 27) + + + +-9056162422514143464->-6847254757107156022 + + + + + +-6847254757107156022->8426695571241120901 + + + + + +-6847256018306484943 + +(36, 27) + + + +2041733466576470103->-6847256018306484943 + + + + + +-6847256018306484943->8426695571241120901 + + + + + +-6847257279505813864 + +(37, 27) + + + +6358163891803576630->-6847257279505813864 + + + + + +-6847257279505813864->8426695571241120901 + + + + + +-6847258540705142785 + +(38, 27) + + + +6970508961356401525->-6847258540705142785 + + + + + +-6847258540705142785->8426695571241120901 + + + + + +-6847259801839228962 + +(39, 27) + + + +-9155011655465785620->-6847259801839228962 + + + + + +-6847259801839228962->8426695571241120901 + + + + + +-6847240883923858283 + +(40, 27) + + + +-7421921102577268021->-6847240883923858283 + + + + + +-6847240883923858283->8426695571241120901 + + + + + +-6847242145123187204 + +(41, 27) + + + +-4517623639816162038->-6847242145123187204 + + + + + +-6847242145123187204->8426695571241120901 + + + + + +-6847243406322516125 + +(42, 27) + + + +-3180879990026077527->-6847243406322516125 + + + + + +-6847243406322516125->8426695571241120901 + + + + + +-6847244667512524654 + +(43, 27) + + + +-8151482191597418720->-6847244667512524654 + + + + + +-6847244667512524654->8426695571241120901 + + + + + +-6847245928711853575 + +(44, 27) + + + +-8720548214153799425->-6847245928711853575 + + + + + +-6847245928711853575->8426695571241120901 + + + + + +-6847247189911182496 + +(45, 27) + + + +-4651457410378817986->-6847247189911182496 + + + + + +-6847247189911182496->8426695571241120901 + + + + + +-6847248451110511417 + +(46, 27) + + + +-3682133972012738691->-6847248451110511417 + + + + + +-6847248451110511417->8426695571241120901 + + + + + +-6847249712319160730 + +(47, 27) + + + +-5954691009816439884->-6847249712319160730 + + + + + +-6847249712319160730->8426695571241120901 + + + + + +-6847230794329226915 + +(48, 27) + + + +913165454815416275->-6847230794329226915 + + + + + +-6847230794329226915->8426695571241120901 + + + + + +-6847291331818956840 + +(0, 28) + + + +-116816442221308298->-6847291331818956840 + + + + + +-76629382402763040 + +simple-shuffle-combine + + + +-6847291331818956840->-76629382402763040 + + + + + +-6847292593015955663 + +(1, 28) + + + +3095107204309032535->-6847292593015955663 + + + + + +-6847292593015955663->-76629382402763040 + + + + + +-6847293854212954486 + +(2, 28) + + + +-7150234107347745320->-6847293854212954486 + + + + + +-6847293854212954486->-76629382402763040 + + + + + +-6847295115419273701 + +(3, 28) + + + +-2674162323368998463->-6847295115419273701 + + + + + +-6847295115419273701->-76629382402763040 + + + + + +-6847296376616272524 + +(4, 28) + + + +-4646311807164342718->-6847296376616272524 + + + + + +-6847296376616272524->-76629382402763040 + + + + + +-6847297637813271347 + +(5, 28) + + + +-3535038096713862621->-6847297637813271347 + + + + + +-6847297637813271347->-76629382402763040 + + + + + +-6847298899010270170 + +(6, 28) + + + +-6753417271959464076->-6847298899010270170 + + + + + +-6847298899010270170->-76629382402763040 + + + + + +-6847300160142026249 + +(7, 28) + + + +-6220399989235439155->-6847300160142026249 + + + + + +-6847300160142026249->-76629382402763040 + + + + + +-6847281242224325472 + +(8, 28) + + + +-7924404998424097154->-6847281242224325472 + + + + + +-6847281242224325472->-76629382402763040 + + + + + +-6847282503421324295 + +(9, 28) + + + +2550528581993434879->-6847282503421324295 + + + + + +-6847282503421324295->-76629382402763040 + + + + + +-6847283764618323118 + +(10, 28) + + + +6046981720907082464->-6847283764618323118 + + + + + +-6847283764618323118->-76629382402763040 + + + + + +-6847285025824642333 + +(11, 28) + + + +-7464465281595259095->-6847285025824642333 + + + + + +-6847285025824642333->-76629382402763040 + + + + + +-6847286287021641156 + +(12, 28) + + + +8945294019369544266->-6847286287021641156 + + + + + +-6847286287021641156->-76629382402763040 + + + + + +-6847287548218639979 + +(13, 28) + + + +-9150873633437163061->-6847287548218639979 + + + + + +-6847287548218639979->-76629382402763040 + + + + + +-6847288809415638802 + +(14, 28) + + + +6925871783075459100->-6847288809415638802 + + + + + +-6847288809415638802->-76629382402763040 + + + + + +-6847290070621958017 + +(15, 28) + + + +-7190127266962414859->-6847290070621958017 + + + + + +-6847290070621958017->-76629382402763040 + + + + + +-6847271152629694104 + +(16, 28) + + + +-2815644721923112986->-6847271152629694104 + + + + + +-6847271152629694104->-76629382402763040 + + + + + +-6847272413826692927 + +(17, 28) + + + +9204043238379769607->-6847272413826692927 + + + + + +-6847272413826692927->-76629382402763040 + + + + + +-6847273675023691750 + +(18, 28) + + + +-4855322530051705208->-6847273675023691750 + + + + + +-6847273675023691750->-76629382402763040 + + + + + +-6847274936230010965 + +(19, 28) + + + +7820185535783727921->-6847274936230010965 + + + + + +-6847274936230010965->-76629382402763040 + + + + + +-6847276197427009788 + +(20, 28) + + + +-1171454758973597006->-6847276197427009788 + + + + + +-6847276197427009788->-76629382402763040 + + + + + +-6847277458624008611 + +(21, 28) + + + +-7366811711852604717->-6847277458624008611 + + + + + +-6847277458624008611->-76629382402763040 + + + + + +-6847278719821007434 + +(22, 28) + + + +7853803715967710244->-6847278719821007434 + + + + + +-6847278719821007434->-76629382402763040 + + + + + +-6847279981027326649 + +(23, 28) + + + +-4879290354035955971->-6847279981027326649 + + + + + +-6847279981027326649->-76629382402763040 + + + + + +-6847261063035062736 + +(24, 28) + + + +5449968583171287022->-6847261063035062736 + + + + + +-6847261063035062736->-76629382402763040 + + + + + +-6847262324232061559 + +(25, 28) + + + +7602287054003199855->-6847262324232061559 + + + + + +-6847262324232061559->-76629382402763040 + + + + + +-6847263585429060382 + +(26, 28) + + + +3493737868525402000->-6847263585429060382 + + + + + +-6847263585429060382->-76629382402763040 + + + + + +-6847264846635379597 + +(27, 28) + + + +295459513295415257->-6847264846635379597 + + + + + +-6847264846635379597->-76629382402763040 + + + + + +-6847266107832378420 + +(28, 28) + + + +-2299114499532796998->-6847266107832378420 + + + + + +-6847266107832378420->-76629382402763040 + + + + + +-6847267369029377243 + +(29, 28) + + + +-139012539128890565->-6847267369029377243 + + + + + +-6847267369029377243->-76629382402763040 + + + + + +-6847268630226376066 + +(30, 28) + + + +-4014047594459862644->-6847268630226376066 + + + + + +-6847268630226376066->-76629382402763040 + + + + + +-6847269891432695281 + +(31, 28) + + + +462290737155496613->-6847269891432695281 + + + + + +-6847269891432695281->-76629382402763040 + + + + + +-6847250973514994504 + +(32, 28) + + + +8861866174736521302->-6847250973514994504 + + + + + +-6847250973514994504->-76629382402763040 + + + + + +-6847252234711993327 + +(33, 28) + + + +-6307418160952923081->-6847252234711993327 + + + + + +-6847252234711993327->-76629382402763040 + + + + + +-6847253495908992150 + +(34, 28) + + + +-2758066477151687496->-6847253495908992150 + + + + + +-6847253495908992150->-76629382402763040 + + + + + +-6847254757115311365 + +(35, 28) + + + +1774676805403264161->-6847254757115311365 + + + + + +-6847254757115311365->-76629382402763040 + + + + + +-6847256018312310188 + +(36, 28) + + + +6303284766562233378->-6847256018312310188 + + + + + +-6847256018312310188->-76629382402763040 + + + + + +-6847257279509309011 + +(37, 28) + + + +-8735842498284990461->-6847257279509309011 + + + + + +-6847257279509309011->-76629382402763040 + + + + + +-6847258540706307834 + +(38, 28) + + + +-536004181321414316->-6847258540706307834 + + + + + +-6847258540706307834->-76629382402763040 + + + + + +-6847259801838063913 + +(39, 28) + + + +7734801968595093421->-6847259801838063913 + + + + + +-6847259801838063913->-76629382402763040 + + + + + +-6847240883920363136 + +(40, 28) + + + +5249734669757464670->-6847240883920363136 + + + + + +-6847240883920363136->-76629382402763040 + + + + + +-6847242145117361959 + +(41, 28) + + + +7400675883626311903->-6847242145117361959 + + + + + +-6847242145117361959->-76629382402763040 + + + + + +-6847243406314360782 + +(42, 28) + + + +-2843168091955794752->-6847243406314360782 + + + + + +-6847243406314360782->-76629382402763040 + + + + + +-6847244667520679997 + +(43, 28) + + + +1567541030633687305->-6847244667520679997 + + + + + +-6847244667520679997->-76629382402763040 + + + + + +-6847245928717678820 + +(44, 28) + + + +-9189075339818862550->-6847245928717678820 + + + + + +-6847245928717678820->-76629382402763040 + + + + + +-6847247189914677643 + +(45, 28) + + + +533337932371156139->-6847247189914677643 + + + + + +-6847247189914677643->-76629382402763040 + + + + + +-6847248451111676466 + +(46, 28) + + + +-2637651623059866116->-6847248451111676466 + + + + + +-6847248451111676466->-76629382402763040 + + + + + +-6847249712317995681 + +(47, 28) + + + +-6989210575974223659->-6847249712317995681 + + + + + +-6847249712317995681->-76629382402763040 + + + + + +-6847230794325731768 + +(48, 28) + + + +8079905664587764166->-6847230794325731768 + + + + + +-6847230794325731768->-76629382402763040 + + + + + +-6847291331820121889 + +(0, 29) + + + +2121253143600782677->-6847291331820121889 + + + + + +1024033431227731435 + +simple-shuffle-combine + + + +-6847291331820121889->1024033431227731435 + + + + + +-6847292593014790614 + +(1, 29) + + + +1618043341579447160->-6847292593014790614 + + + + + +-6847292593014790614->1024033431227731435 + + + + + +-6847293854214119535 + +(2, 29) + + + +-6114609515189039945->-6847293854214119535 + + + + + +-6847293854214119535->1024033431227731435 + + + + + +-6847295115418108652 + +(3, 29) + + + +-4210702340188977438->-6847295115418108652 + + + + + +-6847295115418108652->1024033431227731435 + + + + + +-6847296376617437573 + +(4, 29) + + + +-4864121707070635743->-6847296376617437573 + + + + + +-6847296376617437573->1024033431227731435 + + + + + +-6847297637812106298 + +(5, 29) + + + +-5697725855374722796->-6847297637812106298 + + + + + +-6847297637812106298->1024033431227731435 + + + + + +-6847298899011435219 + +(6, 29) + + + +-4518684953114053501->-6847298899011435219 + + + + + +-6847298899011435219->1024033431227731435 + + + + + +-6847300160140861200 + +(7, 29) + + + +-7827053363325109330->-6847300160140861200 + + + + + +-6847300160140861200->1024033431227731435 + + + + + +-6847281242225490521 + +(8, 29) + + + +1821972620273232221->-6847281242225490521 + + + + + +-6847281242225490521->1024033431227731435 + + + + + +-6847282503420159246 + +(9, 29) + + + +2203096122514021504->-6847282503420159246 + + + + + +-6847282503420159246->1024033431227731435 + + + + + +-6847283764619488167 + +(10, 29) + + + +8755884661202425183->-6847283764619488167 + + + + + +-6847283764619488167->1024033431227731435 + + + + + +-6847285025823477284 + +(11, 29) + + + +-8958206731869314070->-6847285025823477284 + + + + + +-6847285025823477284->1024033431227731435 + + + + + +-6847286287022806205 + +(12, 29) + + + +-8448112129127405175->-6847286287022806205 + + + + + +-6847286287022806205->1024033431227731435 + + + + + +-6847287548217474930 + +(13, 29) + + + +-477095632995908420->-6847287548217474930 + + + + + +-6847287548217474930->1024033431227731435 + + + + + +-6847288809416803851 + +(14, 29) + + + +7176846129019340075->-6847288809416803851 + + + + + +-6847288809416803851->1024033431227731435 + + + + + +-6847290070620792968 + +(15, 29) + + + +9089482247924904982->-6847290070620792968 + + + + + +-6847290070620792968->1024033431227731435 + + + + + +-6847271152630859153 + +(16, 29) + + + +8372780637997478405->-6847271152630859153 + + + + + +-6847271152630859153->1024033431227731435 + + + + + +-6847272413825527878 + +(17, 29) + + + +8869117983326472232->-6847272413825527878 + + + + + +-6847272413825527878->1024033431227731435 + + + + + +-6847273675024856799 + +(18, 29) + + + +-3234428079280084633->-6847273675024856799 + + + + + +-6847273675024856799->1024033431227731435 + + + + + +-6847274936228845916 + +(19, 29) + + + +-1443104012542205870->-6847274936228845916 + + + + + +-6847274936228845916->1024033431227731435 + + + + + +-6847276197428174837 + +(20, 29) + + + +-9204169096674066031->-6847276197428174837 + + + + + +-6847276197428174837->1024033431227731435 + + + + + +-6847277458622843562 + +(21, 29) + + + +-8493435234688108092->-6847277458622843562 + + + + + +-6847277458622843562->1024033431227731435 + + + + + +-6847278719822172483 + +(22, 29) + + + +-8421393668414024397->-6847278719822172483 + + + + + +-6847278719822172483->1024033431227731435 + + + + + +-6847279981026161600 + +(23, 29) + + + +3750144443833431070->-6847279981026161600 + + + + + +-6847279981026161600->1024033431227731435 + + + + + +-6847261063036227785 + +(24, 29) + + + +6503327749672118797->-6847261063036227785 + + + + + +-6847261063036227785->1024033431227731435 + + + + + +-6847262324230896510 + +(25, 29) + + + +4403263036694085680->-6847262324230896510 + + + + + +-6847262324230896510->1024033431227731435 + + + + + +-6847263585430225431 + +(26, 29) + + + +-4631735014295950641->-6847263585430225431 + + + + + +-6847263585430225431->1024033431227731435 + + + + + +-6847264846634214548 + +(27, 29) + + + +-109427391133527718->-6847264846634214548 + + + + + +-6847264846634214548->1024033431227731435 + + + + + +-6847266107833543469 + +(28, 29) + + + +-762577930130593223->-6847266107833543469 + + + + + +-6847266107833543469->1024033431227731435 + + + + + +-6847267369028212194 + +(29, 29) + + + +-1146363209933603284->-6847267369028212194 + + + + + +-6847267369028212194->1024033431227731435 + + + + + +-6847268630227541115 + +(30, 29) + + + +-2894906994614061669->-6847268630227541115 + + + + + +-6847268630227541115->1024033431227731435 + + + + + +-6847269891431530232 + +(31, 29) + + + +7869390623887586438->-6847269891431530232 + + + + + +-6847269891431530232->1024033431227731435 + + + + + +-6847250973516159553 + +(32, 29) + + + +-6837717078832731339->-6847250973516159553 + + + + + +-6847250973516159553->1024033431227731435 + + + + + +-6847252234710828278 + +(33, 29) + + + +-6530675580322969256->-6847252234710828278 + + + + + +-6847252234710828278->1024033431227731435 + + + + + +-6847253495910157199 + +(34, 29) + + + +-2463837668472585321->-6847253495910157199 + + + + + +-6847253495910157199->1024033431227731435 + + + + + +-6847254757114146316 + +(35, 29) + + + +-396869856229233214->-6847254757114146316 + + + + + +-6847254757114146316->1024033431227731435 + + + + + +-6847256018313475237 + +(36, 29) + + + +7356627187908189953->-6847256018313475237 + + + + + +-6847256018313475237->1024033431227731435 + + + + + +-6847257279508143962 + +(37, 29) + + + +444188359904606964->-6847257279508143962 + + + + + +-6847257279508143962->1024033431227731435 + + + + + +-6847258540707472883 + +(38, 29) + + + +-8511067914047203741->-6847258540707472883 + + + + + +-6847258540707472883->1024033431227731435 + + + + + +-6847259801836898864 + +(39, 29) + + + +6753487371146856846->-6847259801836898864 + + + + + +-6847259801836898864->1024033431227731435 + + + + + +-6847240883921528185 + +(40, 29) + + + +5665029654696790845->-6847240883921528185 + + + + + +-6847240883921528185->1024033431227731435 + + + + + +-6847242145116196910 + +(41, 29) + + + +-3610184890931808672->-6847242145116196910 + + + + + +-6847242145116196910->1024033431227731435 + + + + + +-6847243406315525831 + +(42, 29) + + + +-744903597069405377->-6847243406315525831 + + + + + +-6847243406315525831->1024033431227731435 + + + + + +-6847244667519514948 + +(43, 29) + + + +-8279818999500130870->-6847244667519514948 + + + + + +-6847244667519514948->1024033431227731435 + + + + + +-6847245928718843869 + +(44, 29) + + + +560601623826610025->-6847245928718843869 + + + + + +-6847245928718843869->1024033431227731435 + + + + + +-6847247189913512594 + +(45, 29) + + + +-518896131897230436->-6847247189913512594 + + + + + +-6847247189913512594->1024033431227731435 + + + + + +-6847248451112841515 + +(46, 29) + + + +-2279046006932842741->-6847248451112841515 + + + + + +-6847248451112841515->1024033431227731435 + + + + + +-6847249712316830632 + +(47, 29) + + + +-8604685429799873034->-6847249712316830632 + + + + + +-6847249712316830632->1024033431227731435 + + + + + +-6847230794326896817 + +(48, 29) + + + +7375463930247482341->-6847230794326896817 + + + + + +-6847230794326896817->1024033431227731435 + + + + + +-6847291331816626742 + +(0, 30) + + + +-1502186499427375848->-6847291331816626742 + + + + + +-7392883329842454090 + +simple-shuffle-combine + + + +-6847291331816626742->-7392883329842454090 + + + + + +-6847292593018285761 + +(1, 30) + + + +7389775948215132085->-6847292593018285761 + + + + + +-6847292593018285761->-7392883329842454090 + + + + + +-6847293854215284584 + +(2, 30) + + + +-5063387495419885770->-6847293854215284584 + + + + + +-6847293854215284584->-7392883329842454090 + + + + + +-6847295115416943603 + +(3, 30) + + + +-4536068366197158813->-6847295115416943603 + + + + + +-6847295115416943603->-7392883329842454090 + + + + + +-6847296376613942426 + +(4, 30) + + + +-7679215214846478668->-6847296376613942426 + + + + + +-6847296376613942426->-7392883329842454090 + + + + + +-6847297637815601445 + +(5, 30) + + + +-1673133125810617471->-6847297637815601445 + + + + + +-6847297637815601445->-7392883329842454090 + + + + + +-6847298899012600268 + +(6, 30) + + + +-1450263677678646526->-6847298899012600268 + + + + + +-6847298899012600268->-7392883329842454090 + + + + + +-6847300160139696151 + +(7, 30) + + + +-8874857355924774705->-6847300160139696151 + + + + + +-6847300160139696151->-7392883329842454090 + + + + + +-6847281242221995374 + +(8, 30) + + + +213751366659294496->-6847281242221995374 + + + + + +-6847281242221995374->-7392883329842454090 + + + + + +-6847282503423654393 + +(9, 30) + + + +7983857907155101629->-6847282503423654393 + + + + + +-6847282503423654393->-7392883329842454090 + + + + + +-6847283764620653216 + +(10, 30) + + + +9026241064131966014->-6847283764620653216 + + + + + +-6847283764620653216->-7392883329842454090 + + + + + +-6847285025822312235 + +(11, 30) + + + +-1519002932979587829->-6847285025822312235 + + + + + +-6847285025822312235->-7392883329842454090 + + + + + +-6847286287019311058 + +(12, 30) + + + +-2367666422473586084->-6847286287019311058 + + + + + +-6847286287019311058->-7392883329842454090 + + + + + +-6847287548220970077 + +(13, 30) + + + +-7767035994394917911->-6847287548220970077 + + + + + +-6847287548220970077->-7392883329842454090 + + + + + +-6847288809417968900 + +(14, 30) + + + +8716805301432867850->-6847288809417968900 + + + + + +-6847288809417968900->-7392883329842454090 + + + + + +-6847290070619627919 + +(15, 30) + + + +-8569553867429052009->-6847290070619627919 + + + + + +-6847290070619627919->-7392883329842454090 + + + + + +-6847271152627364006 + +(16, 30) + + + +3079101795044967880->-6847271152627364006 + + + + + +-6847271152627364006->-7392883329842454090 + + + + + +-6847272413829023025 + +(17, 30) + + + +2141212568077271141->-6847272413829023025 + + + + + +-6847272413829023025->-7392883329842454090 + + + + + +-6847273675026021848 + +(18, 30) + + + +6510859669978751142->-6847273675026021848 + + + + + +-6847273675026021848->-7392883329842454090 + + + + + +-6847274936227680867 + +(19, 30) + + + +-2415547444643256045->-6847274936227680867 + + + + + +-6847274936227680867->-7392883329842454090 + + + + + +-6847276197424679690 + +(20, 30) + + + +5360537711891593060->-6847276197424679690 + + + + + +-6847276197424679690->-7392883329842454090 + + + + + +-6847277458626338709 + +(21, 30) + + + +2882205737139473649->-6847277458626338709 + + + + + +-6847277458626338709->-7392883329842454090 + + + + + +-6847278719823337532 + +(22, 30) + + + +-9209102716499905422->-6847278719823337532 + + + + + +-6847278719823337532->-7392883329842454090 + + + + + +-6847279981024996551 + +(23, 30) + + + +2275685092606437695->-6847279981024996551 + + + + + +-6847279981024996551->-7392883329842454090 + + + + + +-6847261063032732638 + +(24, 30) + + + +217787584305579472->-6847261063032732638 + + + + + +-6847261063032732638->-7392883329842454090 + + + + + +-6847262324234391657 + +(25, 30) + + + +7870284326644253805->-6847262324234391657 + + + + + +-6847262324234391657->-7392883329842454090 + + + + + +-6847263585431390480 + +(26, 30) + + + +6530060438400159150->-6847263585431390480 + + + + + +-6847263585431390480->-7392883329842454090 + + + + + +-6847264846633049499 + +(27, 30) + + + +-1145050838205948293->-6847264846633049499 + + + + + +-6847264846633049499->-7392883329842454090 + + + + + +-6847266107830048322 + +(28, 30) + + + +5214845501530415052->-6847266107830048322 + + + + + +-6847266107830048322->-7392883329842454090 + + + + + +-6847267369031707341 + +(29, 30) + + + +2788730048942425241->-6847267369031707341 + + + + + +-6847267369031707341->-7392883329842454090 + + + + + +-6847268630228706164 + +(30, 30) + + + +-1438197274491477894->-6847268630228706164 + + + + + +-6847268630228706164->-7392883329842454090 + + + + + +-6847269891430365183 + +(31, 30) + + + +-4915238317501041337->-6847269891430365183 + + + + + +-6847269891430365183->-7392883329842454090 + + + + + +-6847250973512664406 + +(32, 30) + + + +5827429153973487352->-6847250973512664406 + + + + + +-6847250973512664406->-7392883329842454090 + + + + + +-6847252234714323425 + +(33, 30) + + + +-3714063498561703531->-6847252234714323425 + + + + + +-6847252234714323425->-7392883329842454090 + + + + + +-6847253495911322248 + +(34, 30) + + + +-1974592923654291946->-6847253495911322248 + + + + + +-6847253495911322248->-7392883329842454090 + + + + + +-6847254757112981267 + +(35, 30) + + + +-893045997883878589->-6847254757112981267 + + + + + +-6847254757112981267->-7392883329842454090 + + + + + +-6847256018309980090 + +(36, 30) + + + +5978669010785761428->-6847256018309980090 + + + + + +-6847256018309980090->-7392883329842454090 + + + + + +-6847257279511639109 + +(37, 30) + + + +-5519813459986980511->-6847257279511639109 + + + + + +-6847257279511639109->-7392883329842454090 + + + + + +-6847258540708637932 + +(38, 30) + + + +-7537556918931207966->-6847258540708637932 + + + + + +-6847258540708637932->-7392883329842454090 + + + + + +-6847259801835733815 + +(39, 30) + + + +3435745160896529071->-6847259801835733815 + + + + + +-6847259801835733815->-7392883329842454090 + + + + + +-6847240883918033038 + +(40, 30) + + + +-7283245134133396480->-6847240883918033038 + + + + + +-6847240883918033038->-7392883329842454090 + + + + + +-6847242145119692057 + +(41, 30) + + + +399181845804740253->-6847242145119692057 + + + + + +-6847242145119692057->-7392883329842454090 + + + + + +-6847243406316690880 + +(42, 30) + + + +-249799518927800802->-6847243406316690880 + + + + + +-6847243406316690880->-7392883329842454090 + + + + + +-6847244667518349899 + +(43, 30) + + + +-1358537411616673045->-6847244667518349899 + + + + + +-6847244667518349899->-7392883329842454090 + + + + + +-6847245928715348722 + +(44, 30) + + + +-2260418127744424900->-6847245928715348722 + + + + + +-6847245928715348722->-7392883329842454090 + + + + + +-6847247189917007741 + +(45, 30) + + + +1312613996047071689->-6847247189917007741 + + + + + +-6847247189917007741->-7392883329842454090 + + + + + +-6847248451114006564 + +(46, 30) + + + +-44314832047890966->-6847248451114006564 + + + + + +-6847248451114006564->-7392883329842454090 + + + + + +-6847249712315665583 + +(47, 30) + + + +-8907769735030224009->-6847249712315665583 + + + + + +-6847249712315665583->-7392883329842454090 + + + + + +-6847230794323401670 + +(48, 30) + + + +-4685065275656845400->-6847230794323401670 + + + + + +-6847230794323401670->-7392883329842454090 + + + + + +-6847291331817791791 + +(0, 31) + + + +-1090363736461681673->-6847291331817791791 + + + + + +-5175553430421902815 + +simple-shuffle-combine + + + +-6847291331817791791->-5175553430421902815 + + + + + +-6847292593017120712 + +(1, 31) + + + +4142911239066551510->-6847292593017120712 + + + + + +-6847292593017120712->-5175553430421902815 + + + + + +-6847293854216449633 + +(2, 31) + + + +-2955086794830767595->-6847293854216449633 + + + + + +-6847293854216449633->-5175553430421902815 + + + + + +-6847295115415778554 + +(3, 31) + + + +-7211166794714505388->-6847295115415778554 + + + + + +-6847295115415778554->-5175553430421902815 + + + + + +-6847296376615107475 + +(4, 31) + + + +-5445592398147602493->-6847296376615107475 + + + + + +-6847296376615107475->-5175553430421902815 + + + + + +-6847297637814436396 + +(5, 31) + + + +-1526258292810385246->-6847297637814436396 + + + + + +-6847297637814436396->-5175553430421902815 + + + + + +-6847298899013765317 + +(6, 31) + + + +-1667001532951134751->-6847298899013765317 + + + + + +-6847298899013765317->-5175553430421902815 + + + + + +-6847300160138531102 + +(7, 31) + + + +8598339425830696336->-6847300160138531102 + + + + + +-6847300160138531102->-5175553430421902815 + + + + + +-6847281242223160423 + +(8, 31) + + + +-644929106138540129->-6847281242223160423 + + + + + +-6847281242223160423->-5175553430421902815 + + + + + +-6847282503422489344 + +(9, 31) + + + +-4725257084341792546->-6847282503422489344 + + + + + +-6847282503422489344->-5175553430421902815 + + + + + +-6847283764621818265 + +(10, 31) + + + +-6615850267021926627->-6847283764621818265 + + + + + +-6847283764621818265->-5175553430421902815 + + + + + +-6847285025821147186 + +(11, 31) + + + +5268313272325657596->-6847285025821147186 + + + + + +-6847285025821147186->-5175553430421902815 + + + + + +-6847286287020476107 + +(12, 31) + + + +7479689109361827691->-6847286287020476107 + + + + + +-6847286287020476107->-5175553430421902815 + + + + + +-6847287548219805028 + +(13, 31) + + + +-8749442215907449686->-6847287548219805028 + + + + + +-6847287548219805028->-5175553430421902815 + + + + + +-6847288809419133949 + +(14, 31) + + + +-8677401757047780791->-6847288809419133949 + + + + + +-6847288809419133949->-5175553430421902815 + + + + + +-6847290070618462870 + +(15, 31) + + + +8913643126852315832->-6847290070618462870 + + + + + +-6847290070618462870->-5175553430421902815 + + + + + +-6847271152628529055 + +(16, 31) + + + +4201296338089766055->-6847271152628529055 + + + + + +-6847271152628529055->-5175553430421902815 + + + + + +-6847272413827857976 + +(17, 31) + + + +574626964712823366->-6847272413827857976 + + + + + +-6847272413827857976->-5175553430421902815 + + + + + +-6847273675027186897 + +(18, 31) + + + +-308332866379220283->-6847273675027186897 + + + + + +-6847273675027186897->-5175553430421902815 + + + + + +-6847274936226515818 + +(19, 31) + + + +4317678251414868996->-6847274936226515818 + + + + + +-6847274936226515818->-5175553430421902815 + + + + + +-6847276197425844739 + +(20, 31) + + + +-2215100980650576781->-6847276197425844739 + + + + + +-6847276197425844739->-5175553430421902815 + + + + + +-6847277458625173660 + +(21, 31) + + + +1834401664727405074->-6847277458625173660 + + + + + +-6847277458625173660->-5175553430421902815 + + + + + +-6847278719824502581 + +(22, 31) + + + +692518619872159569->-6847278719824502581 + + + + + +-6847278719824502581->-5175553430421902815 + + + + + +-6847279981023831502 + +(23, 31) + + + +-7767941282743909696->-6847279981023831502 + + + + + +-6847279981023831502->-5175553430421902815 + + + + + +-6847261063033897687 + +(24, 31) + + + +1143516597329560847->-6847261063033897687 + + + + + +-6847261063033897687->-5175553430421902815 + + + + + +-6847262324233226608 + +(25, 31) + + + +8645661054729226830->-6847262324233226608 + + + + + +-6847262324233226608->-5175553430421902815 + + + + + +-6847263585432555529 + +(26, 31) + + + +6886368580317797325->-6847263585432555529 + + + + + +-6847263585432555529->-5175553430421902815 + + + + + +-6847264846631884450 + +(27, 31) + + + +5646736293867732332->-6847264846631884450 + + + + + +-6847264846631884450->-5175553430421902815 + + + + + +-6847266107831213371 + +(28, 31) + + + +-3263769937129735973->-6847266107831213371 + + + + + +-6847266107831213371->-5175553430421902815 + + + + + +-6847267369030542292 + +(29, 31) + + + +1407882507375107610->-6847267369030542292 + + + + + +-6847267369030542292->-5175553430421902815 + + + + + +-6847268630229871213 + +(30, 31) + + + +8379127094795577081->-6847268630229871213 + + + + + +-6847268630229871213->-5175553430421902815 + + + + + +-6847269891429200134 + +(31, 31) + + + +-4627734495357634712->-6847269891429200134 + + + + + +-6847269891429200134->-5175553430421902815 + + + + + +-6847250973513829455 + +(32, 31) + + + +8555322941840199127->-6847250973513829455 + + + + + +-6847250973513829455->-5175553430421902815 + + + + + +-6847252234713158376 + +(33, 31) + + + +-4138218592509439306->-6847252234713158376 + + + + + +-6847252234713158376->-5175553430421902815 + + + + + +-6847253495912487297 + +(34, 31) + + + +-931201090398677771->-6847253495912487297 + + + + + +-6847253495912487297->-5175553430421902815 + + + + + +-6847254757111816218 + +(35, 31) + + + +-1795523281237013964->-6847254757111816218 + + + + + +-6847254757111816218->-5175553430421902815 + + + + + +-6847256018311145139 + +(36, 31) + + + +7014257270547600803->-6847256018311145139 + + + + + +-6847256018311145139->-5175553430421902815 + + + + + +-6847257279510474060 + +(37, 31) + + + +-7692468506392367486->-6847257279510474060 + + + + + +-6847257279510474060->-5175553430421902815 + + + + + +-6847258540709802981 + +(38, 31) + + + +-4360808935731555391->-6847258540709802981 + + + + + +-6847258540709802981->-5175553430421902815 + + + + + +-6847259801834568766 + +(39, 31) + + + +2511083606618685296->-6847259801834568766 + + + + + +-6847259801834568766->-5175553430421902815 + + + + + +-6847240883919198087 + +(40, 31) + + + +4204983458625607295->-6847240883919198087 + + + + + +-6847240883919198087->-5175553430421902815 + + + + + +-6847242145118527008 + +(41, 31) + + + +8448613729385682878->-6847242145118527008 + + + + + +-6847242145118527008->-5175553430421902815 + + + + + +-6847243406317855929 + +(42, 31) + + + +-8807116853209517827->-6847243406317855929 + + + + + +-6847243406317855929->-5175553430421902815 + + + + + +-6847244667517184850 + +(43, 31) + + + +-2486301033570282020->-6847244667517184850 + + + + + +-6847244667517184850->-5175553430421902815 + + + + + +-6847245928716513771 + +(44, 31) + + + +-1353510771596742325->-6847245928716513771 + + + + + +-6847245928716513771->-5175553430421902815 + + + + + +-6847247189915842692 + +(45, 31) + + + +1013687424084856714->-6847247189915842692 + + + + + +-6847247189915842692->-5175553430421902815 + + + + + +-6847248451115171613 + +(46, 31) + + + +3076277573217268009->-6847248451115171613 + + + + + +-6847248451115171613->-5175553430421902815 + + + + + +-6847249712314500534 + +(47, 31) + + + +8017120767339515032->-6847249712314500534 + + + + + +-6847249712314500534->-5175553430421902815 + + + + + +-6847230794324566719 + +(48, 31) + + + +7107466729693933191->-6847230794324566719 + + + + + +-6847230794324566719->-5175553430421902815 + + + + + +-6847291331758374292 + +(0, 32) + + + +4864772829922005594->-6847291331758374292 + + + + + +5702302575957607468 + +simple-shuffle-combine + + + +-6847291331758374292->5702302575957607468 + + + + + +-6847292593029936251 + +(1, 32) + + + +-7367348890869690981->-6847292593029936251 + + + + + +-6847292593029936251->5702302575957607468 + + + + + +-6847293854152371938 + +(2, 32) + + + +2150194437379193644->-6847293854152371938 + + + + + +-6847293854152371938->5702302575957607468 + + + + + +-6847295115423933897 + +(3, 32) + + + +1879584913098746637->-6847295115423933897 + + + + + +-6847295115423933897->5702302575957607468 + + + + + +-6847296376546369584 + +(4, 32) + + + +-4737988174649712754->-6847296376546369584 + + + + + +-6847296376546369584->5702302575957607468 + + + + + +-6847297637817931543 + +(5, 32) + + + +3691090192405782479->-6847297637817931543 + + + + + +-6847297637817931543->5702302575957607468 + + + + + +-6847298898940367230 + +(6, 32) + + + +-5307840728950325712->-6847298898940367230 + + + + + +-6847298898940367230->5702302575957607468 + + + + + +-6847300160137366053 + +(7, 32) + + + +6556737475064236161->-6847300160137366053 + + + + + +-6847300160137366053->5702302575957607468 + + + + + +-6847281242182383708 + +(8, 32) + + + +-2886874508824520878->-6847281242182383708 + + + + + +-6847281242182383708->5702302575957607468 + + + + + +-6847282503453945667 + +(9, 32) + + + +4289347433630180147->-6847282503453945667 + + + + + +-6847282503453945667->5702302575957607468 + + + + + +-6847283764576381354 + +(10, 32) + + + +2677271561227039940->-6847283764576381354 + + + + + +-6847283764576381354->5702302575957607468 + + + + + +-6847285025847943313 + +(11, 32) + + + +3203007113379248389->-6847285025847943313 + + + + + +-6847285025847943313->5702302575957607468 + + + + + +-6847286286970379000 + +(12, 32) + + + +-5637290844825921402->-6847286286970379000 + + + + + +-6847286286970379000->5702302575957607468 + + + + + +-6847287548241940959 + +(13, 32) + + + +-1307626031239725977->-6847287548241940959 + + + + + +-6847287548241940959->5702302575957607468 + + + + + +-6847288809364376646 + +(14, 32) + + + +1214822034214673448->-6847288809364376646 + + + + + +-6847288809364376646->5702302575957607468 + + + + + +-6847290070635938605 + +(15, 32) + + + +1878888185578832441->-6847290070635938605 + + + + + +-6847290070635938605->5702302575957607468 + + + + + +-6847271152606393124 + +(16, 32) + + + +7439015692756806890->-6847271152606393124 + + + + + +-6847271152606393124->5702302575957607468 + + + + + +-6847272413877955083 + +(17, 32) + + + +-3624493851078700757->-6847272413877955083 + + + + + +-6847272413877955083->5702302575957607468 + + + + + +-6847273675000390770 + +(18, 32) + + + +-3814093257581505092->-6847273675000390770 + + + + + +-6847273675000390770->5702302575957607468 + + + + + +-6847274936271952729 + +(19, 32) + + + +3496962879308602973->-6847274936271952729 + + + + + +-6847274936271952729->5702302575957607468 + + + + + +-6847276197394388416 + +(20, 32) + + + +-4019355889552495074->-6847276197394388416 + + + + + +-6847276197394388416->5702302575957607468 + + + + + +-6847277458665950375 + +(21, 32) + + + +-4046547654399378849->-6847277458665950375 + + + + + +-6847277458665950375->5702302575957607468 + + + + + +-6847278719788386062 + +(22, 32) + + + +7237619167383541376->-6847278719788386062 + + + + + +-6847278719788386062->5702302575957607468 + + + + + +-6847279981059948021 + +(23, 32) + + + +-3664294335172843631->-6847279981059948021 + + + + + +-6847279981059948021->5702302575957607468 + + + + + +-6847261063030402540 + +(24, 32) + + + +7146408473977021922->-6847261063030402540 + + + + + +-6847261063030402540->5702302575957607468 + + + + + +-6847262324301964499 + +(25, 32) + + + +4324487403022210691->-6847262324301964499 + + + + + +-6847262324301964499->5702302575957607468 + + + + + +-6847263585424400186 + +(26, 32) + + + +-1548728614913080300->-6847263585424400186 + + + + + +-6847263585424400186->5702302575957607468 + + + + + +-6847264846695962145 + +(27, 32) + + + +-1470389414463316907->-6847264846695962145 + + + + + +-6847264846695962145->5702302575957607468 + + + + + +-6847266107818397832 + +(28, 32) + + + +1055394241889975318->-6847266107818397832 + + + + + +-6847266107818397832->5702302575957607468 + + + + + +-6847267369089959791 + +(29, 32) + + + +-2866314360304451657->-6847267369089959791 + + + + + +-6847267369089959791->5702302575957607468 + + + + + +-6847268630212395478 + +(30, 32) + + + +-3313947632707054728->-6847268630212395478 + + + + + +-6847268630212395478->5702302575957607468 + + + + + +-6847269891483957437 + +(31, 32) + + + +-4501686499522562679->-6847269891483957437 + + + + + +-6847269891483957437->5702302575957607468 + + + + + +-6847250973528975092 + +(32, 32) + + + +8416094572138427386->-6847250973528975092 + + + + + +-6847250973528975092->5702302575957607468 + + + + + +-6847252234651410779 + +(33, 32) + + + +4786485202701821883->-6847252234651410779 + + + + + +-6847252234651410779->5702302575957607468 + + + + + +-6847253495922972738 + +(34, 32) + + + +-5587214663671682612->-6847253495922972738 + + + + + +-6847253495922972738->5702302575957607468 + + + + + +-6847254757045408425 + +(35, 32) + + + +-165799759354904275->-6847254757045408425 + + + + + +-6847254757045408425->5702302575957607468 + + + + + +-6847256018316970384 + +(36, 32) + + + +-4172619314115689938->-6847256018316970384 + + + + + +-6847256018316970384->5702302575957607468 + + + + + +-6847257279439406071 + +(37, 32) + + + +642441079958769903->-6847257279439406071 + + + + + +-6847257279439406071->5702302575957607468 + + + + + +-6847258540710968030 + +(38, 32) + + + +-3387261616047514416->-6847258540710968030 + + + + + +-6847258540710968030->5702302575957607468 + + + + + +-6847259801907966853 + +(39, 32) + + + +7682836455031130913->-6847259801907966853 + + + + + +-6847259801907966853->5702302575957607468 + + + + + +-6847240883952984508 + +(40, 32) + + + +7468162837926282738->-6847240883952984508 + + + + + +-6847240883952984508->5702302575957607468 + + + + + +-6847242145075420195 + +(41, 32) + + + +2882449918476062035->-6847242145075420195 + + + + + +-6847242145075420195->5702302575957607468 + + + + + +-6847243406346982154 + +(42, 32) + + + +7235820838286860132->-6847243406346982154 + + + + + +-6847243406346982154->5702302575957607468 + + + + + +-6847244667469417841 + +(43, 32) + + + +-3346629377195984347->-6847244667469417841 + + + + + +-6847244667469417841->5702302575957607468 + + + + + +-6847245928740979800 + +(44, 32) + + + +-846771842470991066->-6847245928740979800 + + + + + +-6847245928740979800->5702302575957607468 + + + + + +-6847247189863415487 + +(45, 32) + + + +-5061980660215774585->-6847247189863415487 + + + + + +-6847247189863415487->5702302575957607468 + + + + + +-6847248451134977446 + +(46, 32) + + + +-3028074845661880632->-6847248451134977446 + + + + + +-6847248451134977446->5702302575957607468 + + + + + +-6847249712257413133 + +(47, 32) + + + +2315628964148566105->-6847249712257413133 + + + + + +-6847249712257413133->5702302575957607468 + + + + + +-6847230794376993924 + +(48, 32) + + + +4231150082615438218->-6847230794376993924 + + + + + +-6847230794376993924->5702302575957607468 + + + + + +-6847291331759539341 + +(0, 33) + + + +-3582671365812353831->-6847291331759539341 + + + + + +6873981168951948743 + +simple-shuffle-combine + + + +-6847291331759539341->6873981168951948743 + + + + + +-6847292593028771202 + +(1, 33) + + + +1101032660309557644->-6847292593028771202 + + + + + +-6847292593028771202->6873981168951948743 + + + + + +-6847293854153536987 + +(2, 33) + + + +-4658566535144544197->-6847293854153536987 + + + + + +-6847293854153536987->6873981168951948743 + + + + + +-6847295115422768848 + +(3, 33) + + + +1393391115968231662->-6847295115422768848 + + + + + +-6847295115422768848->6873981168951948743 + + + + + +-6847296376547534633 + +(4, 33) + + + +-4880416271729936979->-6847296376547534633 + + + + + +-6847296376547534633->6873981168951948743 + + + + + +-6847297637816766494 + +(5, 33) + + + +2202182557052917904->-6847297637816766494 + + + + + +-6847297637816766494->6873981168951948743 + + + + + +-6847298898941532279 + +(6, 33) + + + +-2505520067161621137->-6847298898941532279 + + + + + +-6847298898941532279->6873981168951948743 + + + + + +-6847300160136201004 + +(7, 33) + + + +5583190180858503586->-6847300160136201004 + + + + + +-6847300160136201004->6873981168951948743 + + + + + +-6847281242183548757 + +(8, 33) + + + +-1502706381124752847->-6847281242183548757 + + + + + +-6847281242183548757->6873981168951948743 + + + + + +-6847282503452780618 + +(9, 33) + + + +4502727323944065572->-6847282503452780618 + + + + + +-6847282503452780618->6873981168951948743 + + + + + +-6847283764577546403 + +(10, 33) + + + +4172089594374146515->-6847283764577546403 + + + + + +-6847283764577546403->6873981168951948743 + + + + + +-6847285025846778264 + +(11, 33) + + + +9038576741257715430->-6847285025846778264 + + + + + +-6847285025846778264->6873981168951948743 + + + + + +-6847286286971544049 + +(12, 33) + + + +5441276432305012389->-6847286286971544049 + + + + + +-6847286286971544049->6873981168951948743 + + + + + +-6847287548240775910 + +(13, 33) + + + +-2288922725144735352->-6847287548240775910 + + + + + +-6847287548240775910->6873981168951948743 + + + + + +-6847288809365541695 + +(14, 33) + + + +2763671927018754823->-6847288809365541695 + + + + + +-6847288809365541695->6873981168951948743 + + + + + +-6847290070634773556 + +(15, 33) + + + +8851114979586327482->-6847290070634773556 + + + + + +-6847290070634773556->6873981168951948743 + + + + + +-6847271152607558173 + +(16, 33) + + + +7293245819212519465->-6847271152607558173 + + + + + +-6847271152607558173->6873981168951948743 + + + + + +-6847272413876790034 + +(17, 33) + + + +5004958808158739484->-6847272413876790034 + + + + + +-6847272413876790034->6873981168951948743 + + + + + +-6847273675001555819 + +(18, 33) + + + +-2770719257981304117->-6847273675001555819 + + + + + +-6847273675001555819->6873981168951948743 + + + + + +-6847274936270787680 + +(19, 33) + + + +1885982700875363198->-6847274936270787680 + + + + + +-6847274936270787680->6873981168951948743 + + + + + +-6847276197395553465 + +(20, 33) + + + +-2974855624966756099->-6847276197395553465 + + + + + +-6847276197395553465->6873981168951948743 + + + + + +-6847277458664785326 + +(21, 33) + + + +-5091030013328801824->-6847277458664785326 + + + + + +-6847277458664785326->6873981168951948743 + + + + + +-6847278719789551111 + +(22, 33) + + + +7091852701454749951->-6847278719789551111 + + + + + +-6847278719789551111->6873981168951948743 + + + + + +-6847279981058782972 + +(23, 33) + + + +-5185039469640003406->-6847279981058782972 + + + + + +-6847279981058782972->6873981168951948743 + + + + + +-6847261063031567589 + +(24, 33) + + + +364661246741218497->-6847261063031567589 + + + + + +-6847261063031567589->6873981168951948743 + + + + + +-6847262324300799450 + +(25, 33) + + + +-5564922678874652300->-6847262324300799450 + + + + + +-6847262324300799450->6873981168951948743 + + + + + +-6847263585425565235 + +(26, 33) + + + +-9067726388114526941->-6847263585425565235 + + + + + +-6847263585425565235->6873981168951948743 + + + + + +-6847264846694797096 + +(27, 33) + + + +-3020368940984134282->-6847264846694797096 + + + + + +-6847264846694797096->6873981168951948743 + + + + + +-6847266107819562881 + +(28, 33) + + + +200088534544296693->-6847266107819562881 + + + + + +-6847266107819562881->6873981168951948743 + + + + + +-6847267369088794742 + +(29, 33) + + + +4488555009518871768->-6847267369088794742 + + + + + +-6847267369088794742->6873981168951948743 + + + + + +-6847268630213560527 + +(30, 33) + + + +6432445041713333847->-6847268630213560527 + + + + + +-6847268630213560527->6873981168951948743 + + + + + +-6847269891482792388 + +(31, 33) + + + +-6132971212685552054->-6847269891482792388 + + + + + +-6847269891482792388->6873981168951948743 + + + + + +-6847250973530140141 + +(32, 33) + + + +1056755409582538361->-6847250973530140141 + + + + + +-6847250973530140141->6873981168951948743 + + + + + +-6847252234650245730 + +(33, 33) + + + +3329739152434024108->-6847252234650245730 + + + + + +-6847252234650245730->6873981168951948743 + + + + + +-6847253495924137787 + +(34, 33) + + + +-5233043657452627237->-6847253495924137787 + + + + + +-6847253495924137787->6873981168951948743 + + + + + +-6847254757044243376 + +(35, 33) + + + +-1589291178226014450->-6847254757044243376 + + + + + +-6847254757044243376->6873981168951948743 + + + + + +-6847256018318135433 + +(36, 33) + + + +-4013730910455741363->-6847256018318135433 + + + + + +-6847256018318135433->6873981168951948743 + + + + + +-6847257279438241022 + +(37, 33) + + + +-943408806481454672->-6847257279438241022 + + + + + +-6847257279438241022->6873981168951948743 + + + + + +-6847258540712133079 + +(38, 33) + + + +-2955797899733881841->-6847258540712133079 + + + + + +-6847258540712133079->6873981168951948743 + + + + + +-6847259801906801804 + +(39, 33) + + + +-3809861894988259262->-6847259801906801804 + + + + + +-6847259801906801804->6873981168951948743 + + + + + +-6847240883954149557 + +(40, 33) + + + +-1009345412021200687->-6847240883954149557 + + + + + +-6847240883954149557->6873981168951948743 + + + + + +-6847242145074255146 + +(41, 33) + + + +-6934879017802103740->-6847242145074255146 + + + + + +-6847242145074255146->6873981168951948743 + + + + + +-6847243406348147203 + +(42, 33) + + + +1628845861715352691->-6847243406348147203 + + + + + +-6847243406348147203->6873981168951948743 + + + + + +-6847244667468252792 + +(43, 33) + + + +-4390021202761340922->-6847244667468252792 + + + + + +-6847244667468252792->6873981168951948743 + + + + + +-6847245928742144849 + +(44, 33) + + + +196624594477661509->-6847245928742144849 + + + + + +-6847245928742144849->6873981168951948743 + + + + + +-6847247189862250438 + +(45, 33) + + + +-6732615328011830360->-6847247189862250438 + + + + + +-6847247189862250438->6873981168951948743 + + + + + +-6847248451136142495 + +(46, 33) + + + +-8835516757013226073->-6847248451136142495 + + + + + +-6847248451136142495->6873981168951948743 + + + + + +-6847249712256248084 + +(47, 33) + + + +-7665921284939227686->-6847249712256248084 + + + + + +-6847249712256248084->6873981168951948743 + + + + + +-6847230794378158973 + +(48, 33) + + + +-4478932641559648823->-6847230794378158973 + + + + + +-6847230794378158973->6873981168951948743 + + + + + +-6847291331756044194 + +(0, 34) + + + +-495020488695857556->-6847291331756044194 + + + + + +38999751266899602 + +simple-shuffle-combine + + + +-6847291331756044194->38999751266899602 + + + + + +-6847292593032266349 + +(1, 34) + + + +2810502952843640569->-6847292593032266349 + + + + + +-6847292593032266349->38999751266899602 + + + + + +-6847293854154702036 + +(2, 34) + + + +-3610628689385173222->-6847293854154702036 + + + + + +-6847293854154702036->38999751266899602 + + + + + +-6847295115421603799 + +(3, 34) + + + +53130768217487887->-6847295115421603799 + + + + + +-6847295115421603799->38999751266899602 + + + + + +-6847296376544039486 + +(4, 34) + + + +-8870532717952975504->-6847296376544039486 + + + + + +-6847296376544039486->38999751266899602 + + + + + +-6847297637820261641 + +(5, 34) + + + +5002051313527476429->-6847297637820261641 + + + + + +-6847297637820261641->38999751266899602 + + + + + +-6847298898942697328 + +(6, 34) + + + +-2718937416079421362->-6847298898942697328 + + + + + +-6847298898942697328->38999751266899602 + + + + + +-6847300160135035955 + +(7, 34) + + + +5132444328813202211->-6847300160135035955 + + + + + +-6847300160135035955->38999751266899602 + + + + + +-6847281242180053610 + +(8, 34) + + + +-3229241022986316028->-6847281242180053610 + + + + + +-6847281242180053610->38999751266899602 + + + + + +-6847282503456275765 + +(9, 34) + + + +-8797639653858374319->-6847282503456275765 + + + + + +-6847282503456275765->38999751266899602 + + + + + +-6847283764578711452 + +(10, 34) + + + +-3189365622618828526->-6847283764578711452 + + + + + +-6847283764578711452->38999751266899602 + + + + + +-6847285025845613215 + +(11, 34) + + + +7995198210103942055->-6847285025845613215 + + + + + +-6847285025845613215->38999751266899602 + + + + + +-6847286286968048902 + +(12, 34) + + + +1443006734791125864->-6847286286968048902 + + + + + +-6847286286968048902->38999751266899602 + + + + + +-6847287548244271057 + +(13, 34) + + + +2912615871964907973->-6847287548244271057 + + + + + +-6847287548244271057->38999751266899602 + + + + + +-6847288809366706744 + +(14, 34) + + + +-5865744373252325818->-6847288809366706744 + + + + + +-6847288809366706744->38999751266899602 + + + + + +-6847290070633608507 + +(15, 34) + + + +-1047185744014511909->-6847290070633608507 + + + + + +-6847290070633608507->38999751266899602 + + + + + +-6847271152604063026 + +(16, 34) + + + +2079222347534809340->-6847271152604063026 + + + + + +-6847271152604063026->38999751266899602 + + + + + +-6847272413880285181 + +(17, 34) + + + +-2237030454554810807->-6847272413880285181 + + + + + +-6847272413880285181->38999751266899602 + + + + + +-6847273675002720868 + +(18, 34) + + + +-1862703586879264342->-6847273675002720868 + + + + + +-6847273675002720868->38999751266899602 + + + + + +-6847274936269622631 + +(19, 34) + + + +903576471568651423->-6847274936269622631 + + + + + +-6847274936269622631->38999751266899602 + + + + + +-6847276197392058318 + +(20, 34) + + + +-6612688139068988224->-6847276197392058318 + + + + + +-6847276197392058318->38999751266899602 + + + + + +-6847277458668280473 + +(21, 34) + + + +-1378802478178352099->-6847277458668280473 + + + + + +-6847277458668280473->38999751266899602 + + + + + +-6847278719790716160 + +(22, 34) + + + +8147808806910012126->-6847278719790716160 + + + + + +-6847278719790716160->38999751266899602 + + + + + +-6847279981057617923 + +(23, 34) + + + +-6233111224781098381->-6847279981057617923 + + + + + +-6847279981057617923->38999751266899602 + + + + + +-6847261063028072442 + +(24, 34) + + + +-3066874605156147628->-6847261063028072442 + + + + + +-6847261063028072442->38999751266899602 + + + + + +-6847262324304294597 + +(25, 34) + + + +-8762467892456454175->-6847262324304294597 + + + + + +-6847262324304294597->38999751266899602 + + + + + +-6847263585426730284 + +(26, 34) + + + +900401078621932450->-6847263585426730284 + + + + + +-6847263585426730284->38999751266899602 + + + + + +-6847264846693632047 + +(27, 34) + + + +5605762040156237559->-6847264846693632047 + + + + + +-6847264846693632047->38999751266899602 + + + + + +-6847266107816067734 + +(28, 34) + + + +-3169291610382306632->-6847266107816067734 + + + + + +-6847266107816067734->38999751266899602 + + + + + +-6847267369092289889 + +(29, 34) + + + +8425454301161020693->-6847267369092289889 + + + + + +-6847267369092289889->38999751266899602 + + + + + +-6847268630214725576 + +(30, 34) + + + +8596107376379563222->-6847268630214725576 + + + + + +-6847268630214725576->38999751266899602 + + + + + +-6847269891481627339 + +(31, 34) + + + +1443807629948660587->-6847269891481627339 + + + + + +-6847269891481627339->38999751266899602 + + + + + +-6847250973526644994 + +(32, 34) + + + +7026031302082055436->-6847250973526644994 + + + + + +-6847250973526644994->38999751266899602 + + + + + +-6847252234653740877 + +(33, 34) + + + +6755741194727934233->-6847252234653740877 + + + + + +-6847252234653740877->38999751266899602 + + + + + +-6847253495925302836 + +(34, 34) + + + +-4123061441899131462->-6847253495925302836 + + + + + +-6847253495925302836->38999751266899602 + + + + + +-6847254757043078327 + +(35, 34) + + + +-2632665133503311825->-6847254757043078327 + + + + + +-6847254757043078327->38999751266899602 + + + + + +-6847256018314640286 + +(36, 34) + + + +-7918773461578031088->-6847256018314640286 + + + + + +-6847256018314640286->38999751266899602 + + + + + +-6847257279441736169 + +(37, 34) + + + +4792756547754435053->-6847257279441736169 + + + + + +-6847257279441736169->38999751266899602 + + + + + +-6847258540713298128 + +(38, 34) + + + +-1911315585179323666->-6847258540713298128 + + + + + +-6847258540713298128->38999751266899602 + + + + + +-6847259801905636755 + +(39, 34) + + + +-4854345387918911037->-6847259801905636755 + + + + + +-6847259801905636755->38999751266899602 + + + + + +-6847240883950654410 + +(40, 34) + + + +-6260762785943160412->-6847240883950654410 + + + + + +-6847240883950654410->38999751266899602 + + + + + +-6847242145077750293 + +(41, 34) + + + +-2928854064163331215->-6847242145077750293 + + + + + +-6847242145077750293->38999751266899602 + + + + + +-6847243406349312252 + +(42, 34) + + + +-6929328770047565134->-6847243406349312252 + + + + + +-6847243406349312252->38999751266899602 + + + + + +-6847244667467087743 + +(43, 34) + + + +-5464567676767773497->-6847244667467087743 + + + + + +-6847244667467087743->38999751266899602 + + + + + +-6847245928738649702 + +(44, 34) + + + +7496510940256654600->-6847245928738649702 + + + + + +-6847245928738649702->38999751266899602 + + + + + +-6847247189865745585 + +(45, 34) + + + +5283591948668919781->-6847247189865745585 + + + + + +-6847247189865745585->38999751266899602 + + + + + +-6847248451137307544 + +(46, 34) + + + +-9101144971356438298->-6847248451137307544 + + + + + +-6847248451137307544->38999751266899602 + + + + + +-6847249712255083035 + +(47, 34) + + + +-7519046458590029061->-6847249712255083035 + + + + + +-6847249712255083035->38999751266899602 + + + + + +-6847230794374663826 + +(48, 34) + + + +-7286971967421442148->-6847230794374663826 + + + + + +-6847230794374663826->38999751266899602 + + + + + +-6847291331757209243 + +(0, 35) + + + +-8963669792474626181->-6847291331757209243 + + + + + +-8321258247921784323 + +simple-shuffle-combine + + + +-6847291331757209243->-8321258247921784323 + + + + + +-6847292593031101300 + +(1, 35) + + + +3075022808948815994->-6847292593031101300 + + + + + +-6847292593031101300->-8321258247921784323 + + + + + +-6847293854155867085 + +(2, 35) + + + +-3234273388893510247->-6847293854155867085 + + + + + +-6847293854155867085->-8321258247921784323 + + + + + +-6847295115420438750 + +(3, 35) + + + +-1617370090724445488->-6847295115420438750 + + + + + +-6847295115420438750->-8321258247921784323 + + + + + +-6847296376545204535 + +(4, 35) + + + +-7386091407961800529->-6847296376545204535 + + + + + +-6847296376545204535->-8321258247921784323 + + + + + +-6847297637819096592 + +(5, 35) + + + +4041940690070459054->-6847297637819096592 + + + + + +-6847297637819096592->-8321258247921784323 + + + + + +-6847298898943862377 + +(6, 35) + + + +-986208786096426387->-6847298898943862377 + + + + + +-6847298898943862377->-8321258247921784323 + + + + + +-6847300160133870906 + +(7, 35) + + + +2897713146186031636->-6847300160133870906 + + + + + +-6847300160133870906->-8321258247921784323 + + + + + +-6847281242181218659 + +(8, 35) + + + +-3931357992817846253->-6847281242181218659 + + + + + +-6847281242181218659->-8321258247921784323 + + + + + +-6847282503455110716 + +(9, 35) + + + +-1286392425459809678->-6847282503455110716 + + + + + +-6847282503455110716->-8321258247921784323 + + + + + +-6847283764579876501 + +(10, 35) + + + +8392345446943969265->-6847283764579876501 + + + + + +-6847283764579876501->-8321258247921784323 + + + + + +-6847285025844448166 + +(11, 35) + + + +-2424368759682394936->-6847285025844448166 + + + + + +-6847285025844448166->-8321258247921784323 + + + + + +-6847286286969213951 + +(12, 35) + + + +2992160687078644039->-6847286286969213951 + + + + + +-6847286286969213951->-8321258247921784323 + + + + + +-6847287548243106008 + +(13, 35) + + + +-7952941697042880602->-6847287548243106008 + + + + + +-6847287548243106008->-8321258247921784323 + + + + + +-6847288809367871793 + +(14, 35) + + + +5650881614803919973->-6847288809367871793 + + + + + +-6847288809367871793->-8321258247921784323 + + + + + +-6847290070632443458 + +(15, 35) + + + +-2666971879831775284->-6847290070632443458 + + + + + +-6847290070632443458->-8321258247921784323 + + + + + +-6847271152605228075 + +(16, 35) + + + +6349523891231900171->-6847271152605228075 + + + + + +-6847271152605228075->-8321258247921784323 + + + + + +-6847272413879120132 + +(17, 35) + + + +-3285138533207406582->-6847272413879120132 + + + + + +-6847272413879120132->-8321258247921784323 + + + + + +-6847273675003885917 + +(18, 35) + + + +8747366467418249449->-6847273675003885917 + + + + + +-6847273675003885917->-8321258247921784323 + + + + + +-6847274936268457582 + +(19, 35) + + + +-1129171083479222752->-6847274936268457582 + + + + + +-6847274936268457582->-8321258247921784323 + + + + + +-6847276197393223367 + +(20, 35) + + + +-3810404891492013249->-6847276197393223367 + + + + + +-6847276197393223367->-8321258247921784323 + + + + + +-6847277458667115424 + +(21, 35) + + + +-3613551494460935874->-6847277458667115424 + + + + + +-6847277458667115424->-8321258247921784323 + + + + + +-6847278719791881209 + +(22, 35) + + + +8990366271234247101->-6847278719791881209 + + + + + +-6847278719791881209->-8321258247921784323 + + + + + +-6847279981056452874 + +(23, 35) + + + +1738921079897382244->-6847279981056452874 + + + + + +-6847279981056452874->-8321258247921784323 + + + + + +-6847261063029237491 + +(24, 35) + + + +-2744165197116753053->-6847261063029237491 + + + + + +-6847261063029237491->-8321258247921784323 + + + + + +-6847262324303129548 + +(25, 35) + + + +-222878983567018750->-6847262324303129548 + + + + + +-6847262324303129548->-8321258247921784323 + + + + + +-6847263585427895333 + +(26, 35) + + + +2450359337354308225->-6847263585427895333 + + + + + +-6847263585427895333->-8321258247921784323 + + + + + +-6847264846692466998 + +(27, 35) + + + +-5995199348741216232->-6847264846692466998 + + + + + +-6847264846692466998->-8321258247921784323 + + + + + +-6847266107817232783 + +(28, 35) + + + +8333761803684776343->-6847266107817232783 + + + + + +-6847266107817232783->-8321258247921784323 + + + + + +-6847267369091124840 + +(29, 35) + + + +8634387392422040118->-6847267369091124840 + + + + + +-6847267369091124840->-8321258247921784323 + + + + + +-6847268630215890625 + +(30, 35) + + + +568951548004329397->-6847268630215890625 + + + + + +-6847268630215890625->-8321258247921784323 + + + + + +-6847269891480462290 + +(31, 35) + + + +8089123331206674012->-6847269891480462290 + + + + + +-6847269891480462290->-8321258247921784323 + + + + + +-6847250973527810043 + +(32, 35) + + + +7928777440997782811->-6847250973527810043 + + + + + +-6847250973527810043->-8321258247921784323 + + + + + +-6847252234652575828 + +(33, 35) + + + +-3691023047245661542->-6847252234652575828 + + + + + +-6847252234652575828->-8321258247921784323 + + + + + +-6847253495926467885 + +(34, 35) + + + +-2034764234739948487->-6847253495926467885 + + + + + +-6847253495926467885->-8321258247921784323 + + + + + +-6847254757041913278 + +(35, 35) + + + +-1875020195021194000->-6847254757041913278 + + + + + +-6847254757041913278->-8321258247921784323 + + + + + +-6847256018315805335 + +(36, 35) + + + +-6993044485134734513->-6847256018315805335 + + + + + +-6847256018315805335->-8321258247921784323 + + + + + +-6847257279440571120 + +(37, 35) + + + +2128027107137020878->-6847257279440571120 + + + + + +-6847257279440571120->-8321258247921784323 + + + + + +-6847258540714463177 + +(38, 35) + + + +7838415564540810509->-6847258540714463177 + + + + + +-6847258540714463177->-8321258247921784323 + + + + + +-6847259801904471706 + +(39, 35) + + + +-5269582772672929612->-6847259801904471706 + + + + + +-6847259801904471706->-8321258247921784323 + + + + + +-6847240883951819459 + +(40, 35) + + + +-5216280435933743437->-6847240883951819459 + + + + + +-6847240883951819459->-8321258247921784323 + + + + + +-6847242145076585244 + +(41, 35) + + + +-4485744777246401390->-6847242145076585244 + + + + + +-6847242145076585244->-8321258247921784323 + + + + + +-6847243406350477301 + +(42, 35) + + + +-5881525904798094959->-6847243406350477301 + + + + + +-6847243406350477301->-8321258247921784323 + + + + + +-6847244667465922694 + +(43, 35) + + + +8671026610093107944->-6847244667465922694 + + + + + +-6847244667465922694->-8321258247921784323 + + + + + +-6847245928739814751 + +(44, 35) + + + +-1204948791705634841->-6847245928739814751 + + + + + +-6847245928739814751->-8321258247921784323 + + + + + +-6847247189864580536 + +(45, 35) + + + +4787398001054180806->-6847247189864580536 + + + + + +-6847247189864580536->-8321258247921784323 + + + + + +-6847248451138472593 + +(46, 35) + + + +868108768142418693->-6847248451138472593 + + + + + +-6847248451138472593->-8321258247921784323 + + + + + +-6847249712253917986 + +(47, 35) + + + +8279630784040153580->-6847249712253917986 + + + + + +-6847249712253917986->-8321258247921784323 + + + + + +-6847230794375828875 + +(48, 35) + + + +-6305675238061573973->-6847230794375828875 + + + + + +-6847230794375828875->-8321258247921784323 + + + + + +-6847291331753714096 + +(0, 36) + + + +-3083941708435505906->-6847291331753714096 + + + + + +-7293895496873047048 + +simple-shuffle-combine + + + +-6847291331753714096->-7293895496873047048 + + + + + +-6847292593025276055 + +(1, 36) + + + +-5711058139326396081->-6847292593025276055 + + + + + +-6847292593025276055->-7293895496873047048 + + + + + +-6847293854147711742 + +(2, 36) + + + +-1697612162603083856->-6847293854147711742 + + + + + +-6847293854147711742->-7293895496873047048 + + + + + +-6847295115428594093 + +(3, 36) + + + +7947298445614999737->-6847295115428594093 + + + + + +-6847295115428594093->-7293895496873047048 + + + + + +-6847296376551029780 + +(4, 36) + + + +830191255370134746->-6847296376551029780 + + + + + +-6847296376551029780->-7293895496873047048 + + + + + +-6847297637822591739 + +(5, 36) + + + +8166286810716746779->-6847297637822591739 + + + + + +-6847297637822591739->-7293895496873047048 + + + + + +-6847298898945027426 + +(6, 36) + + + +-762951365652515412->-6847298898945027426 + + + + + +-6847298898945027426->-7293895496873047048 + + + + + +-6847300160132705857 + +(7, 36) + + + +-227312666293160139->-6847300160132705857 + + + + + +-6847300160132705857->-7293895496873047048 + + + + + +-6847281242177723512 + +(8, 36) + + + +-7361806797601257978->-6847281242177723512 + + + + + +-6847281242177723512->-7293895496873047048 + + + + + +-6847282503449285471 + +(9, 36) + + + +-8512897700337289753->-6847282503449285471 + + + + + +-6847282503449285471->-7293895496873047048 + + + + + +-6847283764571721158 + +(10, 36) + + + +-2290822178834908760->-6847283764571721158 + + + + + +-6847283764571721158->-7293895496873047048 + + + + + +-6847285025852603509 + +(11, 36) + + + +7123921663747574289->-6847285025852603509 + + + + + +-6847285025852603509->-7293895496873047048 + + + + + +-6847286286975039196 + +(12, 36) + + + +32314639450770898->-6847286286975039196 + + + + + +-6847286286975039196->-7293895496873047048 + + + + + +-6847287548246601155 + +(13, 36) + + + +-2824261163539161677->-6847287548246601155 + + + + + +-6847287548246601155->-7293895496873047048 + + + + + +-6847288809369036842 + +(14, 36) + + + +6769754488671655748->-6847288809369036842 + + + + + +-6847288809369036842->-7293895496873047048 + + + + + +-6847290070631278409 + +(15, 36) + + + +-3640519129662643059->-6847290070631278409 + + + + + +-6847290070631278409->-7293895496873047048 + + + + + +-6847271152601732928 + +(16, 36) + + + +-8230882730113865826->-6847271152601732928 + + + + + +-6847271152601732928->-7293895496873047048 + + + + + +-6847272413873294887 + +(17, 36) + + + +8751900762420155359->-6847272413873294887 + + + + + +-6847272413873294887->-7293895496873047048 + + + + + +-6847273674995730574 + +(18, 36) + + + +8962771940231885824->-6847273674995730574 + + + + + +-6847273674995730574->-7293895496873047048 + + + + + +-6847274936276612925 + +(19, 36) + + + +8488714598573067273->-6847274936276612925 + + + + + +-6847274936276612925->-7293895496873047048 + + + + + +-6847276197399048612 + +(20, 36) + + + +-5680343258218753174->-6847276197399048612 + + + + + +-6847276197399048612->-7293895496873047048 + + + + + +-6847277458670610571 + +(21, 36) + + + +-761864190480856149->-6847277458670610571 + + + + + +-6847277458670610571->-7293895496873047048 + + + + + +-6847278719793046258 + +(22, 36) + + + +-7227086359397401540->-6847278719793046258 + + + + + +-6847278719793046258->-7293895496873047048 + + + + + +-6847279981055287825 + +(23, 36) + + + +-8077299428393495931->-6847279981055287825 + + + + + +-6847279981055287825->-7293895496873047048 + + + + + +-6847261063025742344 + +(24, 36) + + + +2676746851370505622->-6847261063025742344 + + + + + +-6847261063025742344->-7293895496873047048 + + + + + +-6847262324297304303 + +(25, 36) + + + +467821865704622391->-6847262324297304303 + + + + + +-6847262324297304303->-7293895496873047048 + + + + + +-6847263585419739990 + +(26, 36) + + + +-8381584877766325000->-6847263585419739990 + + + + + +-6847263585419739990->-7293895496873047048 + + + + + +-6847264846700622341 + +(27, 36) + + + +3004521644214168993->-6847264846700622341 + + + + + +-6847264846700622341->-7293895496873047048 + + + + + +-6847266107823058028 + +(28, 36) + + + +5526165357774981218->-6847266107823058028 + + + + + +-6847266107823058028->-7293895496873047048 + + + + + +-6847267369094619987 + +(29, 36) + + + +-4679214365147333373->-6847267369094619987 + + + + + +-6847267369094619987->-7293895496873047048 + + + + + +-6847268630217055674 + +(30, 36) + + + +-6871359481703956844->-6847268630217055674 + + + + + +-6847268630217055674->-7293895496873047048 + + + + + +-6847269891479297241 + +(31, 36) + + + +8315788726486428637->-6847269891479297241 + + + + + +-6847269891479297241->-7293895496873047048 + + + + + +-6847250973524314896 + +(32, 36) + + + +1666220115410464686->-6847250973524314896 + + + + + +-6847250973524314896->-7293895496873047048 + + + + + +-6847252234646750583 + +(33, 36) + + + +-694053293746492817->-6847252234646750583 + + + + + +-6847252234646750583->-7293895496873047048 + + + + + +-6847253495918312542 + +(34, 36) + + + +7233270181628122704->-6847253495918312542 + + + + + +-6847253495918312542->-7293895496873047048 + + + + + +-6847254757050068621 + +(35, 36) + + + +6574893885743492825->-6847254757050068621 + + + + + +-6847254757050068621->-7293895496873047048 + + + + + +-6847256018321630580 + +(36, 36) + + + +-315649725074204038->-6847256018321630580 + + + + + +-6847256018321630580->-7293895496873047048 + + + + + +-6847257279444066267 + +(37, 36) + + + +6320910032015839803->-6847257279444066267 + + + + + +-6847257279444066267->-7293895496873047048 + + + + + +-6847258540715628226 + +(38, 36) + + + +-62681710938120116->-6847258540715628226 + + + + + +-6847258540715628226->-7293895496873047048 + + + + + +-6847259801903306657 + +(39, 36) + + + +2017676642439789013->-6847259801903306657 + + + + + +-6847259801903306657->-7293895496873047048 + + + + + +-6847240883948324312 + +(40, 36) + + + +3067621092943644838->-6847240883948324312 + + + + + +-6847240883948324312->-7293895496873047048 + + + + + +-6847242145070759999 + +(41, 36) + + + +-1680193678458729465->-6847242145070759999 + + + + + +-6847242145070759999->-7293895496873047048 + + + + + +-6847243406342321958 + +(42, 36) + + + +-8398605553940883384->-6847243406342321958 + + + + + +-6847243406342321958->-7293895496873047048 + + + + + +-6847244667474078037 + +(43, 36) + + + +-136390613181930447->-6847244667474078037 + + + + + +-6847244667474078037->-7293895496873047048 + + + + + +-6847245928745639996 + +(44, 36) + + + +5385961300589624434->-6847245928745639996 + + + + + +-6847245928745639996->-7293895496873047048 + + + + + +-6847247189868075683 + +(45, 36) + + + +-7803357671313034285->-6847247189868075683 + + + + + +-6847247189868075683->-7293895496873047048 + + + + + +-6847248451139637642 + +(46, 36) + + + +-4876437806920849948->-6847248451139637642 + + + + + +-6847248451139637642->-7293895496873047048 + + + + + +-6847249712252752937 + +(47, 36) + + + +-2726837458304787795->-6847249712252752937 + + + + + +-6847249712252752937->-7293895496873047048 + + + + + +-6847230794372333728 + +(48, 36) + + + +5804426144232147518->-6847230794372333728 + + + + + +-6847230794372333728->-7293895496873047048 + + + + + +-6847291331754879145 + +(0, 37) + + + +-1471889485299179731->-6847291331754879145 + + + + + +-6197699014181410973 + +simple-shuffle-combine + + + +-6847291331754879145->-6197699014181410973 + + + + + +-6847292593024111006 + +(1, 37) + + + +-6755541633296271856->-6847292593024111006 + + + + + +-6847292593024111006->-6197699014181410973 + + + + + +-6847293854148876791 + +(2, 37) + + + +8118570929155131119->-6847293854148876791 + + + + + +-6847293854148876791->-6197699014181410973 + + + + + +-6847295115427429044 + +(3, 37) + + + +8212926658780424762->-6847295115427429044 + + + + + +-6847295115427429044->-6197699014181410973 + + + + + +-6847296376552194829 + +(4, 37) + + + +-7692747281935040679->-6847296376552194829 + + + + + +-6847296376552194829->-6197699014181410973 + + + + + +-6847297637821426690 + +(5, 37) + + + +7240237983001915404->-6847297637821426690 + + + + + +-6847297637821426690->-6197699014181410973 + + + + + +-6847298898946192475 + +(6, 37) + + + +783587729709451963->-6847298898946192475 + + + + + +-6847298898946192475->-6197699014181410973 + + + + + +-6847300160131540808 + +(7, 37) + + + +-81564023991270314->-6847300160131540808 + + + + + +-6847300160131540808->-6197699014181410973 + + + + + +-6847281242178888561 + +(8, 37) + + + +-4273723370796042203->-6847281242178888561 + + + + + +-6847281242178888561->-6197699014181410973 + + + + + +-6847282503448120422 + +(9, 37) + + + +27795034041502472->-6847282503448120422 + + + + + +-6847282503448120422->-6197699014181410973 + + + + + +-6847283764572886207 + +(10, 37) + + + +-2498664809858590585->-6847283764572886207 + + + + + +-6847283764572886207->-6197699014181410973 + + + + + +-6847285025851438460 + +(11, 37) + + + +-3812683538623662286->-6847285025851438460 + + + + + +-6847285025851438460->-6197699014181410973 + + + + + +-6847286286976204245 + +(12, 37) + + + +-8442841942470446927->-6847286286976204245 + + + + + +-6847286286976204245->-6197699014181410973 + + + + + +-6847287548245436106 + +(13, 37) + + + +4022629880567517348->-6847287548245436106 + + + + + +-6847287548245436106->-6197699014181410973 + + + + + +-6847288809370201891 + +(14, 37) + + + +946347632455041107->-6847288809370201891 + + + + + +-6847288809370201891->-6197699014181410973 + + + + + +-6847290070630113360 + +(15, 37) + + + +4288471382651535982->-6847290070630113360 + + + + + +-6847290070630113360->-6197699014181410973 + + + + + +-6847271152602897977 + +(16, 37) + + + +-7809079340773080451->-6847271152602897977 + + + + + +-6847271152602897977->-6197699014181410973 + + + + + +-6847272413872129838 + +(17, 37) + + + +6517169587500562784->-6847272413872129838 + + + + + +-6847272413872129838->-6197699014181410973 + + + + + +-6847273674996895623 + +(18, 37) + + + +-8436168071007506817->-6847273674996895623 + + + + + +-6847273674996895623->-6197699014181410973 + + + + + +-6847274936275447876 + +(19, 37) + + + +7440910606008042698->-6847274936275447876 + + + + + +-6847274936275447876->-6197699014181410973 + + + + + +-6847276197400213661 + +(20, 37) + + + +4136985642569913001->-6847276197400213661 + + + + + +-6847276197400213661->-6197699014181410973 + + + + + +-6847277458669445522 + +(21, 37) + + + +-616097714523553124->-6847277458669445522 + + + + + +-6847277458669445522->-6197699014181410973 + + + + + +-6847278719794211307 + +(22, 37) + + + +-6806409164097795765->-6847278719794211307 + + + + + +-6847278719794211307->-6197699014181410973 + + + + + +-6847279981054122776 + +(23, 37) + + + +7702078899038430310->-6847279981054122776 + + + + + +-6847279981054122776->-6197699014181410973 + + + + + +-6847261063026907393 + +(24, 37) + + + +-4114982725966597003->-6847261063026907393 + + + + + +-6847261063026907393->-6197699014181410973 + + + + + +-6847262324296139254 + +(25, 37) + + + +136253450494064216->-6847262324296139254 + + + + + +-6847262324296139254->-6197699014181410973 + + + + + +-6847263585420905039 + +(26, 37) + + + +-7408037585777603625->-6847263585420905039 + + + + + +-6847263585420905039->-6197699014181410973 + + + + + +-6847264846699457292 + +(27, 37) + + + +2644812172650817218->-6847264846699457292 + + + + + +-6847264846699457292->-6197699014181410973 + + + + + +-6847266107824223077 + +(28, 37) + + + +5242823852501084993->-6847266107824223077 + + + + + +-6847266107824223077->-6197699014181410973 + + + + + +-6847267369093454938 + +(29, 37) + + + +2840891793935508468->-6847267369093454938 + + + + + +-6847267369093454938->-6197699014181410973 + + + + + +-6847268630218220723 + +(30, 37) + + + +4850273718158627747->-6847268630218220723 + + + + + +-6847268630218220723->-6197699014181410973 + + + + + +-6847269891478132192 + +(31, 37) + + + +7271288541713092862->-6847269891478132192 + + + + + +-6847269891478132192->-6197699014181410973 + + + + + +-6847250973525479945 + +(32, 37) + + + +2574253655692058061->-6847250973525479945 + + + + + +-6847250973525479945->-6197699014181410973 + + + + + +-6847252234645585534 + +(33, 37) + + + +-1767459619860573392->-6847252234645585534 + + + + + +-6847252234645585534->-6197699014181410973 + + + + + +-6847253495919477591 + +(34, 37) + + + +-8456657336398477937->-6847253495919477591 + + + + + +-6847253495919477591->-6197699014181410973 + + + + + +-6847254757048903572 + +(35, 37) + + + +5526992363460981850->-6847254757048903572 + + + + + +-6847254757048903572->-6197699014181410973 + + + + + +-6847256018322795629 + +(36, 37) + + + +1647271122344870137->-6847256018322795629 + + + + + +-6847256018322795629->-6197699014181410973 + + + + + +-6847257279442901218 + +(37, 37) + + + +5837240041724310828->-6847257279442901218 + + + + + +-6847257279442901218->-6197699014181410973 + + + + + +-6847258540716793275 + +(38, 37) + + + +909757216213492059->-6847258540716793275 + + + + + +-6847258540716793275->-6197699014181410973 + + + + + +-6847259801902141608 + +(39, 37) + + + +-7732022688666321162->-6847259801902141608 + + + + + +-6847259801902141608->-6197699014181410973 + + + + + +-6847240883949489361 + +(40, 37) + + + +-5409869252352105787->-6847240883949489361 + + + + + +-6847240883949489361->-6197699014181410973 + + + + + +-6847242145069594950 + +(41, 37) + + + +5803332011965383976->-6847242145069594950 + + + + + +-6847242145069594950->-6197699014181410973 + + + + + +-6847243406343487007 + +(42, 37) + + + +3331886618729191207->-6847243406343487007 + + + + + +-6847243406343487007->-6197699014181410973 + + + + + +-6847244667472912988 + +(43, 37) + + + +-1105374062354319022->-6847244667472912988 + + + + + +-6847244667472912988->-6197699014181410973 + + + + + +-6847245928746805045 + +(44, 37) + + + +-1385564634837282991->-6847245928746805045 + + + + + +-6847245928746805045->-6197699014181410973 + + + + + +-6847247189866910634 + +(45, 37) + + + +6672190069350831812->-6847247189866910634 + + + + + +-6847247189866910634->-6197699014181410973 + + + + + +-6847248451140802691 + +(46, 37) + + + +-3831958854481096973->-6847248451140802691 + + + + + +-6847248451140802691->-6197699014181410973 + + + + + +-6847249712251587888 + +(47, 37) + + + +-2025851282881958770->-6847249712251587888 + + + + + +-6847249712251587888->-6197699014181410973 + + + + + +-6847230794373498777 + +(48, 37) + + + +6781316382054417693->-6847230794373498777 + + + + + +-6847230794373498777->-6197699014181410973 + + + + + +-6847291331751383998 + +(0, 38) + + + +5149482920558123760->-6847291331751383998 + + + + + +4940131213300269102 + +simple-shuffle-combine + + + +-6847291331751383998->4940131213300269102 + + + + + +-6847292593027606153 + +(1, 38) + + + +-364572285135697331->-6847292593027606153 + + + + + +-6847292593027606153->4940131213300269102 + + + + + +-6847293854150041840 + +(2, 38) + + + +-7531293356333532722->-6847293854150041840 + + + + + +-6847293854150041840->4940131213300269102 + + + + + +-6847295115426263995 + +(3, 38) + + + +5095615094179428187->-6847295115426263995 + + + + + +-6847295115426263995->4940131213300269102 + + + + + +-6847296376548699682 + +(4, 38) + + + +-1763140995202902804->-6847296376548699682 + + + + + +-6847296376548699682->4940131213300269102 + + + + + +-6847297637824921837 + +(5, 38) + + + +-7814696393882327687->-6847297637824921837 + + + + + +-6847297637824921837->4940131213300269102 + + + + + +-6847298898947357524 + +(6, 38) + + + +1755994873805821338->-6847298898947357524 + + + + + +-6847298898947357524->4940131213300269102 + + + + + +-6847300160130375759 + +(7, 38) + + + +6780143860165847511->-6847300160130375759 + + + + + +-6847300160130375759->4940131213300269102 + + + + + +-6847281242175393414 + +(8, 38) + + + +-1642911886437672728->-6847281242175393414 + + + + + +-6847281242175393414->4940131213300269102 + + + + + +-6847282503451615569 + +(9, 38) + + + +-7146791884187708603->-6847282503451615569 + + + + + +-6847282503451615569->4940131213300269102 + + + + + +-6847283764574051256 + +(10, 38) + + + +228119519058664390->-6847283764574051256 + + + + + +-6847283764574051256->4940131213300269102 + + + + + +-6847285025850273411 + +(11, 38) + + + +-4865185373210338061->-6847285025850273411 + + + + + +-6847285025850273411->4940131213300269102 + + + + + +-6847286286972709098 + +(12, 38) + + + +4802345080923340164->-6847286286972709098 + + + + + +-6847286286972709098->4940131213300269102 + + + + + +-6847287548248931253 + +(13, 38) + + + +8541916503811896273->-6847287548248931253 + + + + + +-6847287548248931253->4940131213300269102 + + + + + +-6847288809371366940 + +(14, 38) + + + +-7522301705704721518->-6847288809371366940 + + + + + +-6847288809371366940->4940131213300269102 + + + + + +-6847290070628948311 + +(15, 38) + + + +-6791182912737565809->-6847290070628948311 + + + + + +-6847290070628948311->4940131213300269102 + + + + + +-6847271152599402830 + +(16, 38) + + + +-2474530536820928960->-6847271152599402830 + + + + + +-6847271152599402830->4940131213300269102 + + + + + +-6847272413875624985 + +(17, 38) + + + +-6715566062408196707->-6847272413875624985 + + + + + +-6847272413875624985->4940131213300269102 + + + + + +-6847273674998060672 + +(18, 38) + + + +-6960332671088551842->-6847273674998060672 + + + + + +-6847273674998060672->4940131213300269102 + + + + + +-6847274936274282827 + +(19, 38) + + + +6532899476453446123->-6847274936274282827 + + + + + +-6847274936274282827->4940131213300269102 + + + + + +-6847276197396718514 + +(20, 38) + + + +1322705641627562876->-6847276197396718514 + + + + + +-6847276197396718514->4940131213300269102 + + + + + +-6847277458672940669 + +(21, 38) + + + +3026415783360563401->-6847277458672940669 + + + + + +-6847277458672940669->4940131213300269102 + + + + + +-6847278719795376356 + +(22, 38) + + + +-5824002932574072790->-6847278719795376356 + + + + + +-6847278719795376356->4940131213300269102 + + + + + +-6847279981052957727 + +(23, 38) + + + +-2266048574279781081->-6847279981052957727 + + + + + +-6847279981052957727->4940131213300269102 + + + + + +-6847261063023412246 + +(24, 38) + + + +-1598398031460905928->-6847261063023412246 + + + + + +-6847261063023412246->4940131213300269102 + + + + + +-6847262324299634401 + +(25, 38) + + + +-7101458147496154475->-6847262324299634401 + + + + + +-6847262324299634401->4940131213300269102 + + + + + +-6847263585422070088 + +(26, 38) + + + +5439480989815576150->-6847263585422070088 + + + + + +-6847263585422070088->4940131213300269102 + + + + + +-6847264846698292243 + +(27, 38) + + + +-8361620835912909757->-6847264846698292243 + + + + + +-6847264846698292243->4940131213300269102 + + + + + +-6847266107820727930 + +(28, 38) + + + +1244838644930093268->-6847266107820727930 + + + + + +-6847266107820727930->4940131213300269102 + + + + + +-6847267369096950085 + +(29, 38) + + + +5648627103614864993->-6847267369096950085 + + + + + +-6847267369096950085->4940131213300269102 + + + + + +-6847268630219385772 + +(30, 38) + + + +5893652250490188322->-6847268630219385772 + + + + + +-6847268630219385772->4940131213300269102 + + + + + +-6847269891476967143 + +(31, 38) + + + +-3031124702059794913->-6847269891476967143 + + + + + +-6847269891476967143->4940131213300269102 + + + + + +-6847250973521984798 + +(32, 38) + + + +-195684863183038064->-6847250973521984798 + + + + + +-6847250973521984798->4940131213300269102 + + + + + +-6847252234649080681 + +(33, 38) + + + +2285260207771130733->-6847252234649080681 + + + + + +-6847252234649080681->4940131213300269102 + + + + + +-6847253495920642640 + +(34, 38) + + + +-8602422667252175762->-6847253495920642640 + + + + + +-6847253495920642640->4940131213300269102 + + + + + +-6847254757047738523 + +(35, 38) + + + +3600068018425348475->-6847254757047738523 + + + + + +-6847254757047738523->4940131213300269102 + + + + + +-6847256018319300482 + +(36, 38) + + + +-1775678095249513588->-6847256018319300482 + + + + + +-6847256018319300482->4940131213300269102 + + + + + +-6847257279446396365 + +(37, 38) + + + +-9109152310956175463->-6847257279446396365 + + + + + +-6847257279446396365->4940131213300269102 + + + + + +-6847258540717958324 + +(38, 38) + + + +5353749060723455034->-6847258540717958324 + + + + + +-6847258540717958324->4940131213300269102 + + + + + +-6847259801900976559 + +(39, 38) + + + +7958687688092380279->-6847259801900976559 + + + + + +-6847259801900976559->4940131213300269102 + + + + + +-6847240883945994214 + +(40, 38) + + + +7820907074002452104->-6847240883945994214 + + + + + +-6847240883945994214->4940131213300269102 + + + + + +-6847242145073090097 + +(41, 38) + + + +-8558119436664194715->-6847242145073090097 + + + + + +-6847242145073090097->4940131213300269102 + + + + + +-6847243406344652056 + +(42, 38) + + + +-4227142530758939034->-6847243406344652056 + + + + + +-6847243406344652056->4940131213300269102 + + + + + +-6847244667471747939 + +(43, 38) + + + +-1502474091765884397->-6847244667471747939 + + + + + +-6847244667471747939->4940131213300269102 + + + + + +-6847245928743309898 + +(44, 38) + + + +-6585158929665508316->-6847245928743309898 + + + + + +-6847245928743309898->4940131213300269102 + + + + + +-6847247189870405781 + +(45, 38) + + + +-6085118702540977679->-6847247189870405781 + + + + + +-6847247189870405781->4940131213300269102 + + + + + +-6847248451141967740 + +(46, 38) + + + +-4538233602977531598->-6847248451141967740 + + + + + +-6847248451141967740->4940131213300269102 + + + + + +-6847249712250422839 + +(47, 38) + + + +6452765326342785455->-6847249712250422839 + + + + + +-6847249712250422839->4940131213300269102 + + + + + +-6847230794370003630 + +(48, 38) + + + +3193339738691385568->-6847230794370003630 + + + + + +-6847230794370003630->4940131213300269102 + + + + + +-6847291331752549047 + +(0, 39) + + + +5626409877918565935->-6847291331752549047 + + + + + +-3475659347025742023 + +simple-shuffle-combine + + + +-6847291331752549047->-3475659347025742023 + + + + + +-6847292593026441104 + +(1, 39) + + + +-1400213574783537106->-6847292593026441104 + + + + + +-6847292593026441104->-3475659347025742023 + + + + + +-6847293854151206889 + +(2, 39) + + + +-7673715777865084947->-6847293854151206889 + + + + + +-6847293854151206889->-3475659347025742023 + + + + + +-6847295115425098946 + +(3, 39) + + + +2853132207269838412->-6847295115425098946 + + + + + +-6847295115425098946->-3475659347025742023 + + + + + +-6847296376549864731 + +(4, 39) + + + +479340721124773371->-6847296376549864731 + + + + + +-6847296376549864731->-3475659347025742023 + + + + + +-6847297637823756788 + +(5, 39) + + + +9064338590552738554->-6847297637823756788 + + + + + +-6847297637823756788->-3475659347025742023 + + + + + +-6847298898948522573 + +(6, 39) + + + +6164554445352025113->-6847298898948522573 + + + + + +-6847298898948522573->-3475659347025742023 + + + + + +-6847300160129210710 + +(7, 39) + + + +-4218574893834009864->-6847300160129210710 + + + + + +-6847300160129210710->-3475659347025742023 + + + + + +-6847281242176558463 + +(8, 39) + + + +-8480676302685834553->-6847281242176558463 + + + + + +-6847281242176558463->-3475659347025742023 + + + + + +-6847282503450450520 + +(9, 39) + + + +9146138459287609638->-6847282503450450520 + + + + + +-6847282503450450520->-3475659347025742023 + + + + + +-6847283764575216305 + +(10, 39) + + + +-7290855904366273051->-6847283764575216305 + + + + + +-6847283764575216305->-3475659347025742023 + + + + + +-6847285025849108362 + +(11, 39) + + + +-5196785536003959836->-6847285025849108362 + + + + + +-6847285025849108362->-3475659347025742023 + + + + + +-6847286286973874147 + +(12, 39) + + + +5779212872598395539->-6847286286973874147 + + + + + +-6847286286973874147->-3475659347025742023 + + + + + +-6847287548247766204 + +(13, 39) + + + +7001925547217299698->-6847287548247766204 + + + + + +-6847287548247766204->-3475659347025742023 + + + + + +-6847288809372531989 + +(14, 39) + + + +-8233273046902355343->-6847288809372531989 + + + + + +-6847288809372531989->-3475659347025742023 + + + + + +-6847290070627783262 + +(15, 39) + + + +-7834561444999844784->-6847290070627783262 + + + + + +-6847290070627783262->-3475659347025742023 + + + + + +-6847271152600567879 + +(16, 39) + + + +8048744656100737215->-6847271152600567879 + + + + + +-6847271152600567879->-3475659347025742023 + + + + + +-6847272413874459936 + +(17, 39) + + + +-6569822032598108482->-6847272413874459936 + + + + + +-6847272413874459936->-3475659347025742023 + + + + + +-6847273674999225721 + +(18, 39) + + + +3082207792188293949->-6847273674999225721 + + + + + +-6847273674999225721->-3475659347025742023 + + + + + +-6847274936273117778 + +(19, 39) + + + +4461619479762557148->-6847274936273117778 + + + + + +-6847274936273117778->-3475659347025742023 + + + + + +-6847276197397883563 + +(20, 39) + + + +2367187998288013451->-6847276197397883563 + + + + + +-6847276197397883563->-3475659347025742023 + + + + + +-6847277458671775620 + +(21, 39) + + + +214968420569292426->-6847277458671775620 + + + + + +-6847277458671775620->-3475659347025742023 + + + + + +-6847278719796541405 + +(22, 39) + + + +-1442185845864075415->-6847278719796541405 + + + + + +-6847278719796541405->-3475659347025742023 + + + + + +-6847279981051792678 + +(23, 39) + + + +5235231358632037960->-6847279981051792678 + + + + + +-6847279981051792678->-3475659347025742023 + + + + + +-6847261063024577295 + +(24, 39) + + + +-113547293783224553->-6847261063024577295 + + + + + +-6847261063024577295->-3475659347025742023 + + + + + +-6847262324298469352 + +(25, 39) + + + +1515625864989962166->-6847262324298469352 + + + + + +-6847262324298469352->-3475659347025742023 + + + + + +-6847263585423235137 + +(26, 39) + + + +-3108980741532995275->-6847263585423235137 + + + + + +-6847263585423235137->-3475659347025742023 + + + + + +-6847264846697127194 + +(27, 39) + + + +8497739222820415284->-6847264846697127194 + + + + + +-6847264846697127194->-3475659347025742023 + + + + + +-6847266107821892979 + +(28, 39) + + + +-6275263008077484573->-6847266107821892979 + + + + + +-6847266107821892979->-3475659347025742023 + + + + + +-6847267369095785036 + +(29, 39) + + + +-3634730845612547198->-6847267369095785036 + + + + + +-6847267369095785036->-3475659347025742023 + + + + + +-6847268630220550821 + +(30, 39) + + + +6234111350619120897->-6847268630220550821 + + + + + +-6847268630220550821->-3475659347025742023 + + + + + +-6847269891475802094 + +(31, 39) + + + +4256133623357278112->-6847269891475802094 + + + + + +-6847269891475802094->-3475659347025742023 + + + + + +-6847250973523149847 + +(32, 39) + + + +-19100997865757489->-6847250973523149847 + + + + + +-6847250973523149847->-3475659347025742023 + + + + + +-6847252234647915632 + +(33, 39) + + + +-8733507365448062642->-6847252234647915632 + + + + + +-6847252234647915632->-3475659347025742023 + + + + + +-6847253495921807689 + +(34, 39) + + + +-7072801824816521587->-6847253495921807689 + + + + + +-6847253495921807689->-3475659347025742023 + + + + + +-6847254757046573474 + +(35, 39) + + + +2635375086752315500->-6847254757046573474 + + + + + +-6847254757046573474->-3475659347025742023 + + + + + +-6847256018320465531 + +(36, 39) + + + +-1350164750793050213->-6847256018320465531 + + + + + +-6847256018320465531->-3475659347025742023 + + + + + +-6847257279445231316 + +(37, 39) + + + +6607389439887159578->-6847257279445231316 + + + + + +-6847257279445231316->-3475659347025742023 + + + + + +-6847258540719123373 + +(38, 39) + + + +6251800839485582009->-6847258540719123373 + + + + + +-6847258540719123373->-3475659347025742023 + + + + + +-6847259801899811510 + +(39, 39) + + + +-1938509263227371112->-6847259801899811510 + + + + + +-6847259801899811510->-3475659347025742023 + + + + + +-6847240883947159263 + +(40, 39) + + + +8794454331627499879->-6847240883947159263 + + + + + +-6847240883947159263->-3475659347025742023 + + + + + +-6847242145071925048 + +(41, 39) + + + +8918514935029030726->-6847242145071925048 + + + + + +-6847242145071925048->-3475659347025742023 + + + + + +-6847243406345817105 + +(42, 39) + + + +6254522931676724357->-6847243406345817105 + + + + + +-6847243406345817105->-3475659347025742023 + + + + + +-6847244667470582890 + +(43, 39) + + + +6976137968169197828->-6847244667470582890 + + + + + +-6847244667470582890->-3475659347025742023 + + + + + +-6847245928744474947 + +(44, 39) + + + +4421309312696753459->-6847245928744474947 + + + + + +-6847245928744474947->-3475659347025742023 + + + + + +-6847247189869240732 + +(45, 39) + + + +-8030026473099779310->-6847247189869240732 + + + + + +-6847247189869240732->-3475659347025742023 + + + + + +-6847248451143132789 + +(46, 39) + + + +5422144383104101393->-6847248451143132789 + + + + + +-6847248451143132789->-3475659347025742023 + + + + + +-6847249712249257790 + +(47, 39) + + + +3694840401406023280->-6847249712249257790 + + + + + +-6847249712249257790->-3475659347025742023 + + + + + +-6847230794371168679 + +(48, 39) + + + +5426967094772654943->-6847230794371168679 + + + + + +-6847230794371168679->-3475659347025742023 + + + + + +-6847291331767694684 + +(0, 40) + + + +4940929997580244050->-6847291331767694684 + + + + + +-2379458326112179548 + +simple-shuffle-combine + + + +-6847291331767694684->-2379458326112179548 + + + + + +-6847292593039256643 + +(1, 40) + + + +-4596449760938753997->-6847292593039256643 + + + + + +-6847292593039256643->-2379458326112179548 + + + + + +-6847293854161692330 + +(2, 40) + + + +4926497008895487428->-6847293854161692330 + + + + + +-6847293854161692330->-2379458326112179548 + + + + + +-6847295115433254289 + +(3, 40) + + + +-6011189177384195579->-6847295115433254289 + + + + + +-6847295115433254289->-2379458326112179548 + + + + + +-6847296376555689976 + +(4, 40) + + + +5966892323893319046->-6847296376555689976 + + + + + +-6847296376555689976->-2379458326112179548 + + + + + +-6847297637827251935 + +(5, 40) + + + +-4839867122395446937->-6847297637827251935 + + + + + +-6847297637827251935->-2379458326112179548 + + + + + +-6847298898949687622 + +(6, 40) + + + +7207928434975675688->-6847298898949687622 + + + + + +-6847298898949687622->-2379458326112179548 + + + + + +-6847300160128045661 + +(7, 40) + + + +-5190977504125154839->-6847300160128045661 + + + + + +-6847300160128045661->-2379458326112179548 + + + + + +-6847281242173063316 + +(8, 40) + + + +5516473937054446938->-6847281242173063316 + + + + + +-6847281242173063316->-2379458326112179548 + + + + + +-6847282503444625275 + +(9, 40) + + + +-5262540898532318053->-6847282503444625275 + + + + + +-6847282503444625275->-2379458326112179548 + + + + + +-6847283764567060962 + +(10, 40) + + + +-6871161276311330260->-6847283764567060962 + + + + + +-6847283764567060962->-2379458326112179548 + + + + + +-6847285025838622921 + +(11, 40) + + + +497639599935611405->-6847285025838622921 + + + + + +-6847285025838622921->-2379458326112179548 + + + + + +-6847286286961058608 + +(12, 40) + + + +-8348330662068540786->-6847286286961058608 + + + + + +-6847286286961058608->-2379458326112179548 + + + + + +-6847287548232620567 + +(13, 40) + + + +6592883916739257039->-6847287548232620567 + + + + + +-6847287548232620567->-2379458326112179548 + + + + + +-6847288809355056254 + +(14, 40) + + + +-21119354158739152->-6847288809355056254 + + + + + +-6847288809355056254->-2379458326112179548 + + + + + +-6847290070645258997 + +(15, 40) + + + +-7111259237021060975->-6847290070645258997 + + + + + +-6847290070645258997->-2379458326112179548 + + + + + +-6847271152615713516 + +(16, 40) + + + +7944335219741355746->-6847271152615713516 + + + + + +-6847271152615713516->-2379458326112179548 + + + + + +-6847272413887275475 + +(17, 40) + + + +7489333169067194243->-6847272413887275475 + + + + + +-6847272413887275475->-2379458326112179548 + + + + + +-6847273675009711162 + +(18, 40) + + + +8022306480516882708->-6847273675009711162 + + + + + +-6847273675009711162->-2379458326112179548 + + + + + +-6847274936281273121 + +(19, 40) + + + +-4253925299060700843->-6847274936281273121 + + + + + +-6847274936281273121->-2379458326112179548 + + + + + +-6847276197403708808 + +(20, 40) + + + +-1174393604916196074->-6847276197403708808 + + + + + +-6847276197403708808->-2379458326112179548 + + + + + +-6847277458675270767 + +(21, 40) + + + +6026880236742944951->-6847277458675270767 + + + + + +-6847277458675270767->-2379458326112179548 + + + + + +-6847278719797706454 + +(22, 40) + + + +-468638552784168840->-6847278719797706454 + + + + + +-6847278719797706454->-2379458326112179548 + + + + + +-6847279981050627629 + +(23, 40) + + + +2359664826952630585->-6847279981050627629 + + + + + +-6847279981050627629->-2379458326112179548 + + + + + +-6847261063021082148 + +(24, 40) + + + +-4157385808824800278->-6847261063021082148 + + + + + +-6847261063021082148->-2379458326112179548 + + + + + +-6847262324292644107 + +(25, 40) + + + +-5087751021956503509->-6847262324292644107 + + + + + +-6847262324292644107->-2379458326112179548 + + + + + +-6847263585415079794 + +(26, 40) + + + +6745024960371453116->-6847263585415079794 + + + + + +-6847263585415079794->-2379458326112179548 + + + + + +-6847264846686641753 + +(27, 40) + + + +5861727909640463709->-6847264846686641753 + + + + + +-6847264846686641753->-2379458326112179548 + + + + + +-6847266107809077440 + +(28, 40) + + + +-1250157022007712482->-6847266107809077440 + + + + + +-6847266107809077440->-2379458326112179548 + + + + + +-6847267369080639399 + +(29, 40) + + + +-5156341980664311457->-6847267369080639399 + + + + + +-6847267369080639399->-2379458326112179548 + + + + + +-6847268630203075086 + +(30, 40) + + + +-6241317630063833728->-6847268630203075086 + + + + + +-6847268630203075086->-2379458326112179548 + + + + + +-6847269891493277829 + +(31, 40) + + + +4910279657982604321->-6847269891493277829 + + + + + +-6847269891493277829->-2379458326112179548 + + + + + +-6847250973538295484 + +(32, 40) + + + +-8996706360909327630->-6847250973538295484 + + + + + +-6847250973538295484->-2379458326112179548 + + + + + +-6847252234660731171 + +(33, 40) + + + +5965753879828544083->-6847252234660731171 + + + + + +-6847252234660731171->-2379458326112179548 + + + + + +-6847253495932293130 + +(34, 40) + + + +4485944375302847588->-6847253495932293130 + + + + + +-6847253495932293130->-2379458326112179548 + + + + + +-6847254757054728817 + +(35, 40) + + + +-6245288008561155291->-6847254757054728817 + + + + + +-6847254757054728817->-2379458326112179548 + + + + + +-6847256018326290776 + +(36, 40) + + + +5239655478749389862->-6847256018326290776 + + + + + +-6847256018326290776->-2379458326112179548 + + + + + +-6847257279448726463 + +(37, 40) + + + +-8273619780665787513->-6847257279448726463 + + + + + +-6847257279448726463->-2379458326112179548 + + + + + +-6847258540720288422 + +(38, 40) + + + +6756868809984419784->-6847258540720288422 + + + + + +-6847258540720288422->-2379458326112179548 + + + + + +-6847259801898646461 + +(39, 40) + + + +3787220123345672329->-6847259801898646461 + + + + + +-6847259801898646461->-2379458326112179548 + + + + + +-6847240883943664116 + +(40, 40) + + + +4837185910399957754->-6847240883943664116 + + + + + +-6847240883943664116->-2379458326112179548 + + + + + +-6847242145066099803 + +(41, 40) + + + +22860363994725051->-6847242145066099803 + + + + + +-6847242145066099803->-2379458326112179548 + + + + + +-6847243406337661762 + +(42, 40) + + + +-3311249421301353268->-6847243406337661762 + + + + + +-6847243406337661762->-2379458326112179548 + + + + + +-6847244667460097449 + +(43, 40) + + + +4341882099460429869->-6847244667460097449 + + + + + +-6847244667460097449->-2379458326112179548 + + + + + +-6847245928731659408 + +(44, 40) + + + +6410687157526530350->-6847245928731659408 + + + + + +-6847245928731659408->-2379458326112179548 + + + + + +-6847247189854095095 + +(45, 40) + + + +961134422214632431->-6847247189854095095 + + + + + +-6847247189854095095->-2379458326112179548 + + + + + +-6847248451125657054 + +(46, 40) + + + +5786986866200133584->-6847248451125657054 + + + + + +-6847248451125657054->-2379458326112179548 + + + + + +-6847249712266733525 + +(47, 40) + + + +5086662037790259889->-6847249712266733525 + + + + + +-6847249712266733525->-2379458326112179548 + + + + + +-6847230794386314316 + +(48, 40) + + + +5253338724633865602->-6847230794386314316 + + + + + +-6847230794386314316->-2379458326112179548 + + + + + +-6847291331768859733 + +(0, 41) + + + +6460692480019531569->-6847291331768859733 + + + + + +-1353203967595871073 + +simple-shuffle-combine + + + +-6847291331768859733->-1353203967595871073 + + + + + +-6847292593038091594 + +(1, 41) + + + +2271746485097355044->-6847292593038091594 + + + + + +-6847292593038091594->-1353203967595871073 + + + + + +-6847293854162857379 + +(2, 41) + + + +7089148444044846803->-6847293854162857379 + + + + + +-6847293854162857379->-1353203967595871073 + + + + + +-6847295115432089240 + +(3, 41) + + + +-6983649343589705754->-6847295115432089240 + + + + + +-6847295115432089240->-1353203967595871073 + + + + + +-6847296376556855025 + +(4, 41) + + + +-1939623493130329179->-6847296376556855025 + + + + + +-6847296376556855025->-1353203967595871073 + + + + + +-6847297637826086886 + +(5, 41) + + + +-6766893538626728312->-6847297637826086886 + + + + + +-6847297637826086886->-1353203967595871073 + + + + + +-6847298898950852671 + +(6, 41) + + + +8190334666499398663->-6847298898950852671 + + + + + +-6847298898950852671->-1353203967595871073 + + + + + +-6847300160126880612 + +(7, 41) + + + +-4560940270669059414->-6847300160126880612 + + + + + +-6847300160126880612->-1353203967595871073 + + + + + +-6847281242174228365 + +(8, 41) + + + +6986504411599076313->-6847281242174228365 + + + + + +-6847281242174228365->-1353203967595871073 + + + + + +-6847282503443460226 + +(9, 41) + + + +1660625376724558988->-6847282503443460226 + + + + + +-6847282503443460226->-1353203967595871073 + + + + + +-6847283764568226011 + +(10, 41) + + + +-5826678890829734085->-6847283764568226011 + + + + + +-6847283764568226011->-1353203967595871073 + + + + + +-6847285025837457872 + +(11, 41) + + + +-1737109480709578770->-6847285025837457872 + + + + + +-6847285025837457872->-1353203967595871073 + + + + + +-6847286286962223657 + +(12, 41) + + + +2017284929772681389->-6847286286962223657 + + + + + +-6847286286962223657->-1353203967595871073 + + + + + +-6847287548231455518 + +(13, 41) + + + +6171062038668210064->-6847287548231455518 + + + + + +-6847287548231455518->-1353203967595871073 + + + + + +-6847288809356221303 + +(14, 41) + + + +-167994189410623377->-6847288809356221303 + + + + + +-6847288809356221303->-1353203967595871073 + + + + + +-6847290070644093948 + +(15, 41) + + + +-6820167154814940750->-6847290070644093948 + + + + + +-6847290070644093948->-1353203967595871073 + + + + + +-6847271152616878565 + +(16, 41) + + + +367592698401632705->-6847271152616878565 + + + + + +-6847271152616878565->-1353203967595871073 + + + + + +-6847272413886110426 + +(17, 41) + + + +6032587197503294068->-6847272413886110426 + + + + + +-6847272413886110426->-1353203967595871073 + + + + + +-6847273675010876211 + +(18, 41) + + + +7153658738784475683->-6847273675010876211 + + + + + +-6847273675010876211->-1353203967595871073 + + + + + +-6847274936280108072 + +(19, 41) + + + +-6425489856407604618->-6847274936280108072 + + + + + +-6847274936280108072->-1353203967595871073 + + + + + +-6847276197404873857 + +(20, 41) + + + +-8675620445952303115->-6847276197404873857 + + + + + +-6847276197404873857->-1353203967595871073 + + + + + +-6847277458674105718 + +(21, 41) + + + +4439920901329813976->-6847277458674105718 + + + + + +-6847277458674105718->-1353203967595871073 + + + + + +-6847278719798871503 + +(22, 41) + + + +-8424025024117636265->-6847278719798871503 + + + + + +-6847278719798871503->-1353203967595871073 + + + + + +-6847279981049462580 + +(23, 41) + + + +-7490858488354222406->-6847279981049462580 + + + + + +-6847279981049462580->-1353203967595871073 + + + + + +-6847261063022247197 + +(24, 41) + + + +-2567308919195046103->-6847261063022247197 + + + + + +-6847261063022247197->-1353203967595871073 + + + + + +-6847262324291479058 + +(25, 41) + + + +2878453774559801116->-6847262324291479058 + + + + + +-6847262324291479058->-1353203967595871073 + + + + + +-6847263585416244843 + +(26, 41) + + + +6599259629500434891->-6847263585416244843 + + + + + +-6847263585416244843->-1353203967595871073 + + + + + +-6847264846685476704 + +(27, 41) + + + +-3955585988668052866->-6847264846685476704 + + + + + +-6847264846685476704->-1353203967595871073 + + + + + +-6847266107810242489 + +(28, 41) + + + +-202335160297775107->-6847266107810242489 + + + + + +-6847266107810242489->-1353203967595871073 + + + + + +-6847267369079474350 + +(29, 41) + + + +-5655839772651424032->-6847267369079474350 + + + + + +-6847267369079474350->-1353203967595871073 + + + + + +-6847268630204240135 + +(30, 41) + + + +3735668780641649663->-6847268630204240135 + + + + + +-6847268630204240135->-1353203967595871073 + + + + + +-6847269891492112780 + +(31, 41) + + + +3927873435326926146->-6847269891492112780 + + + + + +-6847269891492112780->-1353203967595871073 + + + + + +-6847250973539460533 + +(32, 41) + + + +3833057979921044945->-6847250973539460533 + + + + + +-6847250973539460533->-1353203967595871073 + + + + + +-6847252234659566122 + +(33, 41) + + + +6112660531390839108->-6847252234659566122 + + + + + +-6847252234659566122->-1353203967595871073 + + + + + +-6847253495933458179 + +(34, 41) + + + +5533747301502805363->-6847253495933458179 + + + + + +-6847253495933458179->-1353203967595871073 + + + + + +-6847254757053563768 + +(35, 41) + + + +-7289788262131119866->-6847254757053563768 + + + + + +-6847254757053563768->-1353203967595871073 + + + + + +-6847256018327455825 + +(36, 41) + + + +7237634786726061637->-6847256018327455825 + + + + + +-6847256018327455825->-1353203967595871073 + + + + + +-6847257279447561414 + +(37, 41) + + + +9200721691603343528->-6847257279447561414 + + + + + +-6847257279447561414->-1353203967595871073 + + + + + +-6847258540721453471 + +(38, 41) + + + +8919524777830497959->-6847258540721453471 + + + + + +-6847258540721453471->-1353203967595871073 + + + + + +-6847259801897481412 + +(39, 41) + + + +2743844989761562954->-6847259801897481412 + + + + + +-6847259801897481412->-1353203967595871073 + + + + + +-6847240883944829165 + +(40, 41) + + + +7963351837550974329->-6847240883944829165 + + + + + +-6847240883944829165->-1353203967595871073 + + + + + +-6847242145064934754 + +(41, 41) + + + +-941791631675005524->-6847242145064934754 + + + + + +-6847242145064934754->-1353203967595871073 + + + + + +-6847243406338826811 + +(42, 41) + + + +-2346597353578758693->-6847243406338826811 + + + + + +-6847243406338826811->-1353203967595871073 + + + + + +-6847244667458932400 + +(43, 41) + + + +-7321991368662964722->-6847244667458932400 + + + + + +-6847244667458932400->-1353203967595871073 + + + + + +-6847245928732824457 + +(44, 41) + + + +8033948885261051725->-6847245928732824457 + + + + + +-6847245928732824457->-1353203967595871073 + + + + + +-6847247189852930046 + +(45, 41) + + + +8370274173611549872->-6847247189852930046 + + + + + +-6847247189852930046->-1353203967595871073 + + + + + +-6847248451126822103 + +(46, 41) + + + +-2812401674582819057->-6847248451126822103 + + + + + +-6847248451126822103->-1353203967595871073 + + + + + +-6847249712265568476 + +(47, 41) + + + +-4733951119037603886->-6847249712265568476 + + + + + +-6847249712265568476->-1353203967595871073 + + + + + +-6847230794387479365 + +(48, 41) + + + +8054518117865697377->-6847230794387479365 + + + + + +-6847230794387479365->-1353203967595871073 + + + + + +-6847291331765364586 + +(0, 42) + + + +2507884707565783044->-6847291331765364586 + + + + + +9039872346297589002 + +simple-shuffle-combine + + + +-6847291331765364586->9039872346297589002 + + + + + +-6847292593041586741 + +(1, 42) + + + +-3212615526117206447->-6847292593041586741 + + + + + +-6847292593041586741->9039872346297589002 + + + + + +-6847293854164022428 + +(2, 42) + + + +6459111210588751378->-6847293854164022428 + + + + + +-6847293854164022428->9039872346297589002 + + + + + +-6847295115430924191 + +(3, 42) + + + +-6204794684623338329->-6847295115430924191 + + + + + +-6847295115430924191->9039872346297589002 + + + + + +-6847296376553359878 + +(4, 42) + + + +4011876899052270696->-6847296376553359878 + + + + + +-6847296376553359878->9039872346297589002 + + + + + +-6847297637829582033 + +(5, 42) + + + +7942875876308153029->-6847297637829582033 + + + + + +-6847297637829582033->9039872346297589002 + + + + + +-6847298898952017720 + +(6, 42) + + + +283700088515866438->-6847298898952017720 + + + + + +-6847298898952017720->9039872346297589002 + + + + + +-6847300160125715563 + +(7, 42) + + + +-5533347414765428789->-6847300160125715563 + + + + + +-6847300160125715563->9039872346297589002 + + + + + +-6847281242170733218 + +(8, 42) + + + +-5526168150762708628->-6847281242170733218 + + + + + +-6847281242170733218->9039872346297589002 + + + + + +-6847282503446955373 + +(9, 42) + + + +7466962508328542713->-6847282503446955373 + + + + + +-6847282503446955373->9039872346297589002 + + + + + +-6847283764569391060 + +(10, 42) + + + +-5398117299784551910->-6847283764569391060 + + + + + +-6847283764569391060->9039872346297589002 + + + + + +-6847285025836292823 + +(11, 42) + + + +7518420251712912655->-6847285025836292823 + + + + + +-6847285025836292823->9039872346297589002 + + + + + +-6847286286958728510 + +(12, 42) + + + +-9166144148232219536->-6847286286958728510 + + + + + +-6847286286958728510->9039872346297589002 + + + + + +-6847287548234950665 + +(13, 42) + + + +269190575367459789->-6847287548234950665 + + + + + +-6847287548234950665->9039872346297589002 + + + + + +-6847288809357386352 + +(14, 42) + + + +2940422150941566798->-6847288809357386352 + + + + + +-6847288809357386352->9039872346297589002 + + + + + +-6847290070642928899 + +(15, 42) + + + +-30961657404013709->-6847290070642928899 + + + + + +-6847290070642928899->9039872346297589002 + + + + + +-6847271152613383418 + +(16, 42) + + + +-3575251210567989420->-6847271152613383418 + + + + + +-6847271152613383418->9039872346297589002 + + + + + +-6847272413889605573 + +(17, 42) + + + +685776732316116193->-6847272413889605573 + + + + + +-6847272413889605573->9039872346297589002 + + + + + +-6847273675012041260 + +(18, 42) + + + +-1323849537732501342->-6847273675012041260 + + + + + +-6847273675012041260->9039872346297589002 + + + + + +-6847274936278943023 + +(19, 42) + + + +-5786561912606787593->-6847274936278943023 + + + + + +-6847274936278943023->9039872346297589002 + + + + + +-6847276197401378710 + +(20, 42) + + + +5810904536423553976->-6847276197401378710 + + + + + +-6847276197401378710->9039872346297589002 + + + + + +-6847277458677600865 + +(21, 42) + + + +-8269548395809716715->-6847277458677600865 + + + + + +-6847277458677600865->9039872346297589002 + + + + + +-6847278719800036552 + +(22, 42) + + + +1393266425878615510->-6847278719800036552 + + + + + +-6847278719800036552->9039872346297589002 + + + + + +-6847279981048297531 + +(23, 42) + + + +-7974564874227436581->-6847279981048297531 + + + + + +-6847279981048297531->9039872346297589002 + + + + + +-6847261063018752050 + +(24, 42) + + + +-6680793832097383428->-6847261063018752050 + + + + + +-6847261063018752050->9039872346297589002 + + + + + +-6847262324294974205 + +(25, 42) + + + +8659196527181632841->-6847262324294974205 + + + + + +-6847262324294974205->9039872346297589002 + + + + + +-6847263585417409892 + +(26, 42) + + + +-335540310859985750->-6847263585417409892 + + + + + +-6847263585417409892->9039872346297589002 + + + + + +-6847264846684311655 + +(27, 42) + + + +-5500005809786092641->-6847264846684311655 + + + + + +-6847264846684311655->9039872346297589002 + + + + + +-6847266107806747342 + +(28, 42) + + + +4510661565374261184->-6847266107806747342 + + + + + +-6847266107806747342->9039872346297589002 + + + + + +-6847267369082969497 + +(29, 42) + + + +203469232576772893->-6847267369082969497 + + + + + +-6847267369082969497->9039872346297589002 + + + + + +-6847268630205405184 + +(30, 42) + + + +-877939447792454178->-6847268630205405184 + + + + + +-6847268630205405184->9039872346297589002 + + + + + +-6847269891490947731 + +(31, 42) + + + +2391337958911243971->-6847269891490947731 + + + + + +-6847269891490947731->9039872346297589002 + + + + + +-6847250973535965386 + +(32, 42) + + + +-2008478922490617180->-6847250973535965386 + + + + + +-6847250973535965386->9039872346297589002 + + + + + +-6847252234663061269 + +(33, 42) + + + +136539045823458417->-6847252234663061269 + + + + + +-6847252234663061269->9039872346297589002 + + + + + +-6847253495934623228 + +(34, 42) + + + +-1332125240756931662->-6847253495934623228 + + + + + +-6847253495934623228->9039872346297589002 + + + + + +-6847254757052398719 + +(35, 42) + + + +-427846397552864825->-6847254757052398719 + + + + + +-6847254757052398719->9039872346297589002 + + + + + +-6847256018323960678 + +(36, 42) + + + +2699504060371567112->-6847256018323960678 + + + + + +-6847256018323960678->9039872346297589002 + + + + + +-6847257279451056561 + +(37, 42) + + + +7250192934157287653->-6847257279451056561 + + + + + +-6847257279451056561->9039872346297589002 + + + + + +-6847258540722618520 + +(38, 42) + + + +-8653548083575584282->-6847258540722618520 + + + + + +-6847258540722618520->9039872346297589002 + + + + + +-6847259801896316363 + +(39, 42) + + + +3519239621389763179->-6847259801896316363 + + + + + +-6847259801896316363->9039872346297589002 + + + + + +-6847240883941334018 + +(40, 42) + + + +-6736736392440602612->-6847240883941334018 + + + + + +-6847240883941334018->9039872346297589002 + + + + + +-6847242145068429901 + +(41, 42) + + + +5382654817774283801->-6847242145068429901 + + + + + +-6847242145068429901->9039872346297589002 + + + + + +-6847243406339991860 + +(42, 42) + + + +8585480326162129082->-6847243406339991860 + + + + + +-6847243406339991860->9039872346297589002 + + + + + +-6847244667457767351 + +(43, 42) + + + +-8366473760812914897->-6847244667457767351 + + + + + +-6847244667457767351->9039872346297589002 + + + + + +-6847245928729329310 + +(44, 42) + + + +4493640861487062800->-6847245928729329310 + + + + + +-6847245928729329310->9039872346297589002 + + + + + +-6847247189856425193 + +(45, 42) + + + +-4777252700452064019->-6847247189856425193 + + + + + +-6847247189856425193->9039872346297589002 + + + + + +-6847248451127987152 + +(46, 42) + + + +-578777687284708882->-6847248451127987152 + + + + + +-6847248451127987152->9039872346297589002 + + + + + +-6847249712264403427 + +(47, 42) + + + +-6340586559619754861->-6847249712264403427 + + + + + +-6847249712264403427->9039872346297589002 + + + + + +-6847230794383984218 + +(48, 42) + + + +2274079379209291252->-6847230794383984218 + + + + + +-6847230794383984218->9039872346297589002 + + + + + +-6847291331766529635 + +(0, 43) + + + +3888695933329177875->-6847291331766529635 + + + + + +692935105895691477 + +simple-shuffle-combine + + + +-6847291331766529635->692935105895691477 + + + + + +-6847292593040421692 + +(1, 43) + + + +-3544215734376878222->-6847292593040421692 + + + + + +-6847292593040421692->692935105895691477 + + + + + +-6847293854165187477 + +(2, 43) + + + +-2164862795734232847->-6847293854165187477 + + + + + +-6847293854165187477->692935105895691477 + + + + + +-6847295115429759142 + +(3, 43) + + + +1301986149686189512->-6847295115429759142 + + + + + +-6847295115429759142->692935105895691477 + + + + + +-6847296376554524927 + +(4, 43) + + + +4919088262600947271->-6847296376554524927 + + + + + +-6847296376554524927->692935105895691477 + + + + + +-6847297637828416984 + +(5, 43) + + + +-3787633058144380762->-6847297637828416984 + + + + + +-6847297637828416984->692935105895691477 + + + + + +-6847298898953182769 + +(6, 43) + + + +-8416669703028930203->-6847298898953182769 + + + + + +-6847298898953182769->692935105895691477 + + + + + +-6847300160124550514 + +(7, 43) + + + +-8732371510795760964->-6847300160124550514 + + + + + +-6847300160124550514->692935105895691477 + + + + + +-6847281242171898267 + +(8, 43) + + + +4441959315921789563->-6847281242171898267 + + + + + +-6847281242171898267->692935105895691477 + + + + + +-6847282503445790324 + +(9, 43) + + + +4635759870517250938->-6847282503445790324 + + + + + +-6847282503445790324->692935105895691477 + + + + + +-6847283764570556109 + +(10, 43) + + + +5490707045731776665->-6847283764570556109 + + + + + +-6847283764570556109->692935105895691477 + + + + + +-6847285025835127774 + +(11, 43) + + + +6399550792077065680->-6847285025835127774 + + + + + +-6847285025835127774->692935105895691477 + + + + + +-6847286286959893559 + +(12, 43) + + + +9134833457262247855->-6847286286959893559 + + + + + +-6847286286959893559->692935105895691477 + + + + + +-6847287548233785616 + +(13, 43) + + + +7482041614087210414->-6847287548233785616 + + + + + +-6847287548233785616->692935105895691477 + + + + + +-6847288809358551401 + +(14, 43) + + + +5174067341804075373->-6847288809358551401 + + + + + +-6847288809358551401->692935105895691477 + + + + + +-6847290070641763850 + +(15, 43) + + + +7328342324561609316->-6847290070641763850 + + + + + +-6847290070641763850->692935105895691477 + + + + + +-6847271152614548467 + +(16, 43) + + + +-2537313552999353245->-6847271152614548467 + + + + + +-6847271152614548467->692935105895691477 + + + + + +-6847272413888440524 + +(17, 43) + + + +8533812120450402818->-6847272413888440524 + + + + + +-6847272413888440524->692935105895691477 + + + + + +-6847273675013206309 + +(18, 43) + + + +-8826201469829395583->-6847273675013206309 + + + + + +-6847273675013206309->692935105895691477 + + + + + +-6847274936277777974 + +(19, 43) + + + +-6711218917612289768->-6847274936277777974 + + + + + +-6847274936277777974->692935105895691477 + + + + + +-6847276197402543759 + +(20, 43) + + + +5546421004903744151->-6847276197402543759 + + + + + +-6847276197402543759->692935105895691477 + + + + + +-6847277458676435816 + +(21, 43) + + + +7070254227527062326->-6847277458676435816 + + + + + +-6847277458676435816->692935105895691477 + + + + + +-6847278719801201601 + +(22, 43) + + + +3490440397308207285->-6847278719801201601 + + + + + +-6847278719801201601->692935105895691477 + + + + + +-6847279981047132482 + +(23, 43) + + + +654891153758522060->-6847279981047132482 + + + + + +-6847279981047132482->692935105895691477 + + + + + +-6847261063019917099 + +(24, 43) + + + +-5632989839515038453->-6847261063019917099 + + + + + +-6847261063019917099->692935105895691477 + + + + + +-6847262324293809156 + +(25, 43) + + + +-4105344799300825334->-6847262324293809156 + + + + + +-6847262324293809156->692935105895691477 + + + + + +-6847263585418574941 + +(26, 43) + + + +-8813048551974065175->-6847263585418574941 + + + + + +-6847263585418574941->692935105895691477 + + + + + +-6847264846683146606 + +(27, 43) + + + +2968607171537719584->-6847264846683146606 + + + + + +-6847264846683146606->692935105895691477 + + + + + +-6847266107807912391 + +(28, 43) + + + +5562894465763728959->-6847266107807912391 + + + + + +-6847266107807912391->692935105895691477 + + + + + +-6847267369081804448 + +(29, 43) + + + +5866675987922767934->-6847267369081804448 + + + + + +-6847267369081804448->692935105895691477 + + + + + +-6847268630206570233 + +(30, 43) + + + +9091318751155150013->-6847268630206570233 + + + + + +-6847268630206570233->692935105895691477 + + + + + +-6847269891489782682 + +(31, 43) + + + +-7346164057177584204->-6847269891489782682 + + + + + +-6847269891489782682->692935105895691477 + + + + + +-6847250973537130435 + +(32, 43) + + + +-448263056214704205->-6847250973537130435 + + + + + +-6847250973537130435->692935105895691477 + + + + + +-6847252234661896220 + +(33, 43) + + + +-911265016612059758->-6847252234661896220 + + + + + +-6847252234661896220->692935105895691477 + + + + + +-6847253495935788277 + +(34, 43) + + + +-8780183401053531503->-6847253495935788277 + + + + + +-6847253495935788277->692935105895691477 + + + + + +-6847254757051233670 + +(35, 43) + + + +8120329155616103400->-6847254757051233670 + + + + + +-6847254757051233670->692935105895691477 + + + + + +-6847256018325125727 + +(36, 43) + + + +4240603372917907687->-6847256018325125727 + + + + + +-6847256018325125727->692935105895691477 + + + + + +-6847257279449891512 + +(37, 43) + + + +6206796497208635078->-6847257279449891512 + + + + + +-6847257279449891512->692935105895691477 + + + + + +-6847258540723783569 + +(38, 43) + + + +-7605727356906099707->-6847258540723783569 + + + + + +-6847258540723783569->692935105895691477 + + + + + +-6847259801895151314 + +(39, 43) + + + +-6448887844186229412->-6847259801895151314 + + + + + +-6847259801895151314->692935105895691477 + + + + + +-6847240883942499067 + +(40, 43) + + + +2603558518881149979->-6847240883942499067 + + + + + +-6847240883942499067->692935105895691477 + + + + + +-6847242145067264852 + +(41, 43) + + + +4400247452214691226->-6847242145067264852 + + + + + +-6847242145067264852->692935105895691477 + + + + + +-6847243406341156909 + +(42, 43) + + + +-8256160720556508359->-6847243406341156909 + + + + + +-6847243406341156909->692935105895691477 + + + + + +-6847244667456602302 + +(43, 43) + + + +-859656533121392656->-6847244667456602302 + + + + + +-6847244667456602302->692935105895691477 + + + + + +-6847245928730494359 + +(44, 43) + + + +-3406637168494165425->-6847245928730494359 + + + + + +-6847245928730494359->692935105895691477 + + + + + +-6847247189855260144 + +(45, 43) + + + +3687924390134515406->-6847247189855260144 + + + + + +-6847247189855260144->692935105895691477 + + + + + +-6847248451129152201 + +(46, 43) + + + +-219051527100267507->-6847248451129152201 + + + + + +-6847248451129152201->692935105895691477 + + + + + +-6847249712263238378 + +(47, 43) + + + +2138030048513804164->-6847249712263238378 + + + + + +-6847249712263238378->692935105895691477 + + + + + +-6847230794385149267 + +(48, 43) + + + +-5692125415090002173->-6847230794385149267 + + + + + +-6847230794385149267->692935105895691477 + + + + + +-6847291331763034488 + +(0, 44) + + + +-8996934411831836922->-6847291331763034488 + + + + + +1789118194496842352 + +simple-shuffle-combine + + + +-6847291331763034488->1789118194496842352 + + + + + +-6847292593034596447 + +(1, 44) + + + +-2927905435481088281->-6847292593034596447 + + + + + +-6847292593034596447->1789118194496842352 + + + + + +-6847293854157032134 + +(2, 44) + + + +6734962443719282344->-6847293854157032134 + + + + + +-6847293854157032134->1789118194496842352 + + + + + +-6847295115437914485 + +(3, 44) + + + +-7597539544118692079->-6847295115437914485 + + + + + +-6847295115437914485->1789118194496842352 + + + + + +-6847296376560350172 + +(4, 44) + + + +2074188407509403346->-6847296376560350172 + + + + + +-6847296376560350172->1789118194496842352 + + + + + +-6847297637831912131 + +(5, 44) + + + +1467256573327312563->-6847297637831912131 + + + + + +-6847297637831912131->1789118194496842352 + + + + + +-6847298898954347818 + +(6, 44) + + + +-6759457844188343228->-6847298898954347818 + + + + + +-6847298898954347818->1789118194496842352 + + + + + +-6847300160123385465 + +(7, 44) + + + +8670998572129673277->-6847300160123385465 + + + + + +-6847300160123385465->1789118194496842352 + + + + + +-6847281242168403120 + +(8, 44) + + + +383377347104769038->-6847281242168403120 + + + + + +-6847281242168403120->1789118194496842352 + + + + + +-6847282503439965079 + +(9, 44) + + + +7698963518950006863->-6847282503439965079 + + + + + +-6847282503439965079->1789118194496842352 + + + + + +-6847283764562400766 + +(10, 44) + + + +-5284564422600705360->-6847283764562400766 + + + + + +-6847283764562400766->1789118194496842352 + + + + + +-6847285025843283117 + +(11, 44) + + + +5015905947640216505->-6847285025843283117 + + + + + +-6847285025843283117->1789118194496842352 + + + + + +-6847286286965718804 + +(12, 44) + + + +-3934504491752526886->-6847286286965718804 + + + + + +-6847286286965718804->1789118194496842352 + + + + + +-6847287548237280763 + +(13, 44) + + + +-5672677791096911077->-6847287548237280763 + + + + + +-6847287548237280763->1789118194496842352 + + + + + +-6847288809359716450 + +(14, 44) + + + +-3913823280536525652->-6847288809359716450 + + + + + +-6847288809359716450->1789118194496842352 + + + + + +-6847290070640598801 + +(15, 44) + + + +8124945462860427141->-6847290070640598801 + + + + + +-6847290070640598801->1789118194496842352 + + + + + +-6847271152611053320 + +(16, 44) + + + +-7800222625764625770->-6847271152611053320 + + + + + +-6847271152611053320->1789118194496842352 + + + + + +-6847272413882615279 + +(17, 44) + + + +1934437038139033143->-6847272413882615279 + + + + + +-6847272413882615279->1789118194496842352 + + + + + +-6847273675005050966 + +(18, 44) + + + +-798608303081992->-6847273675005050966 + + + + + +-6847273675005050966->1789118194496842352 + + + + + +-6847274936285933317 + +(19, 44) + + + +2853449653125469857->-6847274936285933317 + + + + + +-6847274936285933317->1789118194496842352 + + + + + +-6847276197408369004 + +(20, 44) + + + +-4670431581761669790->-6847276197408369004 + + + + + +-6847276197408369004->1789118194496842352 + + + + + +-6847277458679930963 + +(21, 44) + + + +-6754682678991431165->-6847277458679930963 + + + + + +-6847277458679930963->1789118194496842352 + + + + + +-6847278719802366650 + +(22, 44) + + + +2777121893051066260->-6847278719802366650 + + + + + +-6847278719802366650->1789118194496842352 + + + + + +-6847279981045967433 + +(23, 44) + + + +-2084212103199564915->-6847279981045967433 + + + + + +-6847279981045967433->1789118194496842352 + + + + + +-6847261063016421952 + +(24, 44) + + + +8619747142748577438->-6847261063016421952 + + + + + +-6847261063016421952->1789118194496842352 + + + + + +-6847262324287983911 + +(25, 44) + + + +-1190613698393195809->-6847262324287983911 + + + + + +-6847262324287983911->1789118194496842352 + + + + + +-6847263585410419598 + +(26, 44) + + + +-8855212455170848000->-6847263585410419598 + + + + + +-6847263585410419598->1789118194496842352 + + + + + +-6847264846691301949 + +(27, 44) + + + +-6919856353729398007->-6847264846691301949 + + + + + +-6847264846691301949->1789118194496842352 + + + + + +-6847266107813737636 + +(28, 44) + + + +2606508516418381418->-6847266107813737636 + + + + + +-6847266107813737636->1789118194496842352 + + + + + +-6847267369085299595 + +(29, 44) + + + +-8500707880073302357->-6847267369085299595 + + + + + +-6847267369085299595->1789118194496842352 + + + + + +-6847268630207735282 + +(30, 44) + + + +-9088965591186271428->-6847268630207735282 + + + + + +-6847268630207735282->1789118194496842352 + + + + + +-6847269891488617633 + +(31, 44) + + + +1132451414720414421->-6847269891488617633 + + + + + +-6847269891488617633->1789118194496842352 + + + + + +-6847250973533635288 + +(32, 44) + + + +-6158795463319703130->-6847250973533635288 + + + + + +-6847250973533635288->1789118194496842352 + + + + + +-6847252234656070975 + +(33, 44) + + + +-9097669494407351033->-6847252234656070975 + + + + + +-6847252234656070975->1789118194496842352 + + + + + +-6847253495927632934 + +(34, 44) + + + +1236728470315663688->-6847253495927632934 + + + + + +-6847253495927632934->1789118194496842352 + + + + + +-6847254757059389013 + +(35, 44) + + + +-611593700592490191->-6847254757059389013 + + + + + +-6847254757059389013->1789118194496842352 + + + + + +-6847256018330950972 + +(36, 44) + + + +-6007555571052570254->-6847256018330950972 + + + + + +-6847256018330950972->1789118194496842352 + + + + + +-6847257279453386659 + +(37, 44) + + + +9173164231443771603->-6847257279453386659 + + + + + +-6847257279453386659->1789118194496842352 + + + + + +-6847258540724948618 + +(38, 44) + + + +-7822428860954447132->-6847258540724948618 + + + + + +-6847258540724948618->1789118194496842352 + + + + + +-6847259801893986265 + +(39, 44) + + + +-613300310582164771->-6847259801893986265 + + + + + +-6847259801893986265->1789118194496842352 + + + + + +-6847240883939003920 + +(40, 44) + + + +-827154606533803346->-6847240883939003920 + + + + + +-6847240883939003920->1789118194496842352 + + + + + +-6847242145061439607 + +(41, 44) + + + +-4949662311517704849->-6847242145061439607 + + + + + +-6847242145061439607->1789118194496842352 + + + + + +-6847243406333001566 + +(42, 44) + + + +-644298177854817968->-6847243406333001566 + + + + + +-6847243406333001566->1789118194496842352 + + + + + +-6847244667464757645 + +(43, 44) + + + +7626544253432657369->-6847244667464757645 + + + + + +-6847244667464757645->1789118194496842352 + + + + + +-6847245928736319604 + +(44, 44) + + + +-4026004857989407366->-6847245928736319604 + + + + + +-6847245928736319604->1789118194496842352 + + + + + +-6847247189858755291 + +(45, 44) + + + +7631888851942998331->-6847247189858755291 + + + + + +-6847247189858755291->1789118194496842352 + + + + + +-6847248451130317250 + +(46, 44) + + + +-8697663585944164532->-6847248451130317250 + + + + + +-6847248451130317250->1789118194496842352 + + + + + +-6847249712262073329 + +(47, 44) + + + +607322113550106789->-6847249712262073329 + + + + + +-6847249712262073329->1789118194496842352 + + + + + +-6847230794381654120 + +(48, 44) + + + +-1901834945792586698->-6847230794381654120 + + + + + +-6847230794381654120->1789118194496842352 + + + + + +-6847291331764199537 + +(0, 45) + + + +-9222472477943357147->-6847291331764199537 + + + + + +2815390449853383227 + +simple-shuffle-combine + + + +-6847291331764199537->2815390449853383227 + + + + + +-6847292593033431398 + +(1, 45) + + + +3853878086445070344->-6847292593033431398 + + + + + +-6847292593033431398->2815390449853383227 + + + + + +-6847293854158197183 + +(2, 45) + + + +2125483606212852103->-6847293854158197183 + + + + + +-6847293854158197183->2815390449853383227 + + + + + +-6847295115436749436 + +(3, 45) + + + +-750648501137839054->-6847295115436749436 + + + + + +-6847295115436749436->2815390449853383227 + + + + + +-6847296376561515221 + +(4, 45) + + + +-3752540159990166095->-6847296376561515221 + + + + + +-6847296376561515221->2815390449853383227 + + + + + +-6847297637830747082 + +(5, 45) + + + +493709288006945188->-6847297637830747082 + + + + + +-6847297637830747082->2815390449853383227 + + + + + +-6847298898955512867 + +(6, 45) + + + +-6976159420306875053->-6847298898955512867 + + + + + +-6847298898955512867->2815390449853383227 + + + + + +-6847300160122220416 + +(7, 45) + + + +-2864485405130455714->-6847300160122220416 + + + + + +-6847300160122220416->2815390449853383227 + + + + + +-6847281242169568169 + +(8, 45) + + + +1426773748581242413->-6847281242169568169 + + + + + +-6847281242169568169->2815390449853383227 + + + + + +-6847282503438800030 + +(9, 45) + + + +-4004555745761116912->-6847282503438800030 + + + + + +-6847282503438800030->2815390449853383227 + + + + + +-6847283764563565815 + +(10, 45) + + + +6442472264278645231->-6847283764563565815 + + + + + +-6847283764563565815->2815390449853383227 + + + + + +-6847285025842118068 + +(11, 45) + + + +-6666808845557017286->-6847285025842118068 + + + + + +-6847285025842118068->2815390449853383227 + + + + + +-6847286286966883853 + +(12, 45) + + + +399628203620032089->-6847286286966883853 + + + + + +-6847286286966883853->2815390449853383227 + + + + + +-6847287548236115714 + +(13, 45) + + + +-5609320180705912052->-6847287548236115714 + + + + + +-6847287548236115714->2815390449853383227 + + + + + +-6847288809360881499 + +(14, 45) + + + +-2865982884630276677->-6847288809360881499 + + + + + +-6847288809360881499->2815390449853383227 + + + + + +-6847290070639433752 + +(15, 45) + + + +7148059801988497766->-6847290070639433752 + + + + + +-6847290070639433752->2815390449853383227 + + + + + +-6847271152612218369 + +(16, 45) + + + +2106954949713205621->-6847271152612218369 + + + + + +-6847271152612218369->2815390449853383227 + + + + + +-6847272413881450230 + +(17, 45) + + + +-9027672942637425832->-6847272413881450230 + + + + + +-6847272413881450230->2815390449853383227 + + + + + +-6847273675006216015 + +(18, 45) + + + +2678726406776020183->-6847273675006216015 + + + + + +-6847273675006216015->2815390449853383227 + + + + + +-6847274936284768268 + +(19, 45) + + + +-8086477262763053118->-6847274936284768268 + + + + + +-6847274936284768268->2815390449853383227 + + + + + +-6847276197409534053 + +(20, 45) + + + +-1855833945508935615->-6847276197409534053 + + + + + +-6847276197409534053->2815390449853383227 + + + + + +-6847277458678765914 + +(21, 45) + + + +-7226173234478326540->-6847277458678765914 + + + + + +-6847277458678765914->2815390449853383227 + + + + + +-6847278719803531699 + +(22, 45) + + + +3740665601496400035->-6847278719803531699 + + + + + +-6847278719803531699->2815390449853383227 + + + + + +-6847279981044802384 + +(23, 45) + + + +-1938462336942278290->-6847279981044802384 + + + + + +-6847279981044802384->2815390449853383227 + + + + + +-6847261063017587001 + +(24, 45) + + + +1170598450161775997->-6847261063017587001 + + + + + +-6847261063017587001->2815390449853383227 + + + + + +-6847262324286818862 + +(25, 45) + + + +-2168456074941665184->-6847262324286818862 + + + + + +-6847262324286818862->2815390449853383227 + + + + + +-6847263585411584647 + +(26, 45) + + + +425832416470392191->-6847263585411584647 + + + + + +-6847263585411584647->2815390449853383227 + + + + + +-6847264846690136900 + +(27, 45) + + + +519311121683468234->-6847264846690136900 + + + + + +-6847264846690136900->2815390449853383227 + + + + + +-6847266107814902685 + +(28, 45) + + + +-4243801692167211607->-6847266107814902685 + + + + + +-6847266107814902685->2815390449853383227 + + + + + +-6847267369084134546 + +(29, 45) + + + +-22091270831237732->-6847267369084134546 + + + + + +-6847267369084134546->2815390449853383227 + + + + + +-6847268630208900331 + +(30, 45) + + + +1918607630619676747->-6847268630208900331 + + + + + +-6847268630208900331->2815390449853383227 + + + + + +-6847269891487452584 + +(31, 45) + + + +643226287160365046->-6847269891487452584 + + + + + +-6847269891487452584->2815390449853383227 + + + + + +-6847250973534800337 + +(32, 45) + + + +5492764680258667461->-6847250973534800337 + + + + + +-6847250973534800337->2815390449853383227 + + + + + +-6847252234654905926 + +(33, 45) + + + +8999332439806279208->-6847252234654905926 + + + + + +-6847252234654905926->2815390449853383227 + + + + + +-6847253495928797983 + +(34, 45) + + + +944528019964237863->-6847253495928797983 + + + + + +-6847253495928797983->2815390449853383227 + + + + + +-6847254757058223964 + +(35, 45) + + + +-3424268336802523566->-6847254757058223964 + + + + + +-6847254757058223964->2815390449853383227 + + + + + +-6847256018332116021 + +(36, 45) + + + +-4971932151692789679->-6847256018332116021 + + + + + +-6847256018332116021->2815390449853383227 + + + + + +-6847257279452221610 + +(37, 45) + + + +6388408506220249028->-6847257279452221610 + + + + + +-6847257279452221610->2815390449853383227 + + + + + +-6847258540726113667 + +(38, 45) + + + +-5012393971223327757->-6847258540726113667 + + + + + +-6847258540726113667->2815390449853383227 + + + + + +-6847259801892821216 + +(39, 45) + + + +-1661125649674169346->-6847259801892821216 + + + + + +-6847259801892821216->2815390449853383227 + + + + + +-6847240883940168969 + +(40, 45) + + + +-7781219920774151987->-6847240883940168969 + + + + + +-6847240883940168969->2815390449853383227 + + + + + +-6847242145060274558 + +(41, 45) + + + +-4239758393559388624->-6847242145060274558 + + + + + +-6847242145060274558->2815390449853383227 + + + + + +-6847243406334166615 + +(42, 45) + + + +430248289500581007->-6847243406334166615 + + + + + +-6847243406334166615->2815390449853383227 + + + + + +-6847244667463592596 + +(43, 45) + + + +-2772668563804768422->-6847244667463592596 + + + + + +-6847244667463592596->2815390449853383227 + + + + + +-6847245928737484653 + +(44, 45) + + + +5875617615744044025->-6847245928737484653 + + + + + +-6847245928737484653->2815390449853383227 + + + + + +-6847247189857590242 + +(45, 45) + + + +6667236794196954156->-6847247189857590242 + + + + + +-6847247189857590242->2815390449853383227 + + + + + +-6847248451131482299 + +(46, 45) + + + +2778388888606999643->-6847248451131482299 + + + + + +-6847248451131482299->2815390449853383227 + + + + + +-6847249712260908280 + +(47, 45) + + + +8047611868749636230->-6847249712260908280 + + + + + +-6847249712260908280->2815390449853383227 + + + + + +-6847230794382819169 + +(48, 45) + + + +1199532914036262677->-6847230794382819169 + + + + + +-6847230794382819169->2815390449853383227 + + + + + +-6847291331760704390 + +(0, 46) + + + +-2046131356769234456->-6847291331760704390 + + + + + +-4562113501990289914 + +simple-shuffle-combine + + + +-6847291331760704390->-4562113501990289914 + + + + + +-6847292593036926545 + +(1, 46) + + + +-8877775409583625147->-6847292593036926545 + + + + + +-6847292593036926545->-4562113501990289914 + + + + + +-6847293854159362232 + +(2, 46) + + + +-6352023500899998522->-6847293854159362232 + + + + + +-6847293854159362232->-4562113501990289914 + + + + + +-6847295115435584387 + +(3, 46) + + + +-1729866935177033229->-6847295115435584387 + + + + + +-6847295115435584387->-4562113501990289914 + + + + + +-6847296376558020074 + +(4, 46) + + + +7867737022545227396->-6847296376558020074 + + + + + +-6847296376558020074->-4562113501990289914 + + + + + +-6847297637834242229 + +(5, 46) + + + +2851090807040354513->-6847297637834242229 + + + + + +-6847297637834242229->-4562113501990289914 + + + + + +-6847298898956677916 + +(6, 46) + + + +4758632420524321938->-6847298898956677916 + + + + + +-6847298898956677916->-4562113501990289914 + + + + + +-6847300160121055367 + +(7, 46) + + + +5761341498825030527->-6847300160121055367 + + + + + +-6847300160121055367->-4562113501990289914 + + + + + +-6847281242166073022 + +(8, 46) + + + +7255394766097493488->-6847281242166073022 + + + + + +-6847281242166073022->-4562113501990289914 + + + + + +-6847282503442295177 + +(9, 46) + + + +678219146343982413->-6847282503442295177 + + + + + +-6847282503442295177->-4562113501990289914 + + + + + +-6847283764564730864 + +(10, 46) + + + +6290162162280269006->-6847283764564730864 + + + + + +-6847283764564730864->-4562113501990289914 + + + + + +-6847285025840953019 + +(11, 46) + + + +-6448962624356234661->-6847285025840953019 + + + + + +-6847285025840953019->-4562113501990289914 + + + + + +-6847286286963388706 + +(12, 46) + + + +4188831590262032364->-6847286286963388706 + + + + + +-6847286286963388706->-4562113501990289914 + + + + + +-6847287548239610861 + +(13, 46) + + + +-1501214846549582727->-6847287548239610861 + + + + + +-6847287548239610861->-4562113501990289914 + + + + + +-6847288809362046548 + +(14, 46) + + + +8618924021180934298->-6847288809362046548 + + + + + +-6847288809362046548->-4562113501990289914 + + + + + +-6847290070638268703 + +(15, 46) + + + +-3859516798525198809->-6847290070638268703 + + + + + +-6847290070638268703->-4562113501990289914 + + + + + +-6847271152608723222 + +(16, 46) + + + +-9179630265242974920->-6847271152608723222 + + + + + +-6847271152608723222->-4562113501990289914 + + + + + +-6847272413884945377 + +(17, 46) + + + +-3784704254675328107->-6847272413884945377 + + + + + +-6847272413884945377->-4562113501990289914 + + + + + +-6847273675007381064 + +(18, 46) + + + +-5798781835377283242->-6847273675007381064 + + + + + +-6847273675007381064->-4562113501990289914 + + + + + +-6847274936283603219 + +(19, 46) + + + +-2501878474183893693->-6847274936283603219 + + + + + +-6847274936283603219->-4562113501990289914 + + + + + +-6847276197406038906 + +(20, 46) + + + +-7631977627352874540->-6847276197406038906 + + + + + +-6847276197406038906->-4562113501990289914 + + + + + +-6847277458682261061 + +(21, 46) + + + +-4292265175838125215->-6847277458682261061 + + + + + +-6847277458682261061->-4562113501990289914 + + + + + +-6847278719804696748 + +(22, 46) + + + +6984052447520991010->-6847278719804696748 + + + + + +-6847278719804696748->-4562113501990289914 + + + + + +-6847279981043637335 + +(23, 46) + + + +3394165401896321679->-6847279981043637335 + + + + + +-6847279981043637335->-4562113501990289914 + + + + + +-6847261063014091854 + +(24, 46) + + + +7216627392361786688->-6847261063014091854 + + + + + +-6847261063014091854->-4562113501990289914 + + + + + +-6847262324290314009 + +(25, 46) + + + +-8049292526000265059->-6847262324290314009 + + + + + +-6847262324290314009->-4562113501990289914 + + + + + +-6847263585412749696 + +(26, 46) + + + +1472929622296854366->-6847263585412749696 + + + + + +-6847263585412749696->-4562113501990289914 + + + + + +-6847264846688971851 + +(27, 46) + + + +7227812450224743659->-6847264846688971851 + + + + + +-6847264846688971851->-4562113501990289914 + + + + + +-6847266107811407538 + +(28, 46) + + + +-8759632828547001732->-6847266107811407538 + + + + + +-6847266107811407538->-4562113501990289914 + + + + + +-6847267369087629693 + +(29, 46) + + + +5246200011168488393->-6847267369087629693 + + + + + +-6847267369087629693->-4562113501990289914 + + + + + +-6847268630210065380 + +(30, 46) + + + +3596992805512103722->-6847268630210065380 + + + + + +-6847268630210065380->-4562113501990289914 + + + + + +-6847269891486287535 + +(31, 46) + + + +-400147703623752329->-6847269891486287535 + + + + + +-6847269891486287535->-4562113501990289914 + + + + + +-6847250973531305190 + +(32, 46) + + + +-7563801960004265080->-6847250973531305190 + + + + + +-6847250973531305190->-4562113501990289914 + + + + + +-6847252234658401073 + +(33, 46) + + + +-3720140440859318683->-6847252234658401073 + + + + + +-6847252234658401073->-4562113501990289914 + + + + + +-6847253495929963032 + +(34, 46) + + + +-5819501332670225562->-6847253495929963032 + + + + + +-6847253495929963032->-4562113501990289914 + + + + + +-6847254757057058915 + +(35, 46) + + + +5272798853073958675->-6847254757057058915 + + + + + +-6847254757057058915->-4562113501990289914 + + + + + +-6847256018328620874 + +(36, 46) + + + +8214503783364024612->-6847256018328620874 + + + + + +-6847256018328620874->-4562113501990289914 + + + + + +-6847257279455716757 + +(37, 46) + + + +-7907460042387336463->-6847257279455716757 + + + + + +-6847257279455716757->-4562113501990289914 + + + + + +-6847258540727278716 + +(38, 46) + + + +-3597475316028549582->-6847258540727278716 + + + + + +-6847258540727278716->-4562113501990289914 + + + + + +-6847259801891656167 + +(39, 46) + + + +-1444424117913181921->-6847259801891656167 + + + + + +-6847259801891656167->-4562113501990289914 + + + + + +-6847240883936673822 + +(40, 46) + + + +6105928136805058704->-6847240883936673822 + + + + + +-6847240883936673822->-4562113501990289914 + + + + + +-6847242145063769705 + +(41, 46) + + + +-794916797531626899->-6847242145063769705 + + + + + +-6847242145063769705->-4562113501990289914 + + + + + +-6847243406335331664 + +(42, 46) + + + +3617386551753827182->-6847243406335331664 + + + + + +-6847243406335331664->-4562113501990289914 + + + + + +-6847244667462427547 + +(43, 46) + + + +-3817150912723000197->-6847244667462427547 + + + + + +-6847244667462427547->-4562113501990289914 + + + + + +-6847245928733989506 + +(44, 46) + + + +9078432369254377100->-6847245928733989506 + + + + + +-6847245928733989506->-4562113501990289914 + + + + + +-6847247189861085389 + +(45, 46) + + + +703267963432022681->-6847247189861085389 + + + + + +-6847247189861085389->-4562113501990289914 + + + + + +-6847248451132647348 + +(46, 46) + + + +3826192958802057018->-6847248451132647348 + + + + + +-6847248451132647348->-4562113501990289914 + + + + + +-6847249712259743231 + +(47, 46) + + + +7670152749488931655->-6847249712259743231 + + + + + +-6847249712259743231->-4562113501990289914 + + + + + +-6847230794379324022 + +(48, 46) + + + +-4002002313216356648->-6847230794379324022 + + + + + +-6847230794379324022->-4562113501990289914 + + + + + +-6847291331761869439 + +(0, 47) + + + +7709348316417150919->-6847291331761869439 + + + + + +5532963792721227377 + +simple-shuffle-combine + + + +-6847291331761869439->5532963792721227377 + + + + + +-6847292593035761496 + +(1, 47) + + + +8082992868573344294->-6847292593035761496 + + + + + +-6847292593035761496->5532963792721227377 + + + + + +-6847293854160527281 + +(2, 47) + + + +-4881972920246155547->-6847293854160527281 + + + + + +-6847293854160527281->5532963792721227377 + + + + + +-6847295115434419338 + +(3, 47) + + + +-4333622006463199004->-6847295115434419338 + + + + + +-6847295115434419338->5532963792721227377 + + + + + +-6847296376559185123 + +(4, 47) + + + +-7902799139345038445->-6847296376559185123 + + + + + +-6847296376559185123->5532963792721227377 + + + + + +-6847297637833077180 + +(5, 47) + + + +1877543522811172338->-6847297637833077180 + + + + + +-6847297637833077180->5532963792721227377 + + + + + +-6847298898957842965 + +(6, 47) + + + +-2751506485177666703->-6847298898957842965 + + + + + +-6847298898957842965->5532963792721227377 + + + + + +-6847300160119890318 + +(7, 47) + + + +4145887991457079552->-6847300160119890318 + + + + + +-6847300160119890318->5532963792721227377 + + + + + +-6847281242167238071 + +(8, 47) + + + +-1223217293854909137->-6847281242167238071 + + + + + +-6847281242167238071->5532963792721227377 + + + + + +-6847282503441130128 + +(9, 47) + + + +8663651900224872238->-6847282503441130128 + + + + + +-6847282503441130128->5532963792721227377 + + + + + +-6847283764565895913 + +(10, 47) + + + +7260271865405100781->-6847283764565895913 + + + + + +-6847283764565895913->5532963792721227377 + + + + + +-6847285025839787970 + +(11, 47) + + + +-7502340241846176436->-6847285025839787970 + + + + + +-6847285025839787970->5532963792721227377 + + + + + +-6847286286964553755 + +(12, 47) + + + +-4911373419663142661->-6847286286964553755 + + + + + +-6847286286964553755->5532963792721227377 + + + + + +-6847287548238445812 + +(13, 47) + + + +5909033236342673914->-6847287548238445812 + + + + + +-6847287548238445812->5532963792721227377 + + + + + +-6847288809363211597 + +(14, 47) + + + +-8754413790638579943->-6847288809363211597 + + + + + +-6847288809363211597->5532963792721227377 + + + + + +-6847290070637103654 + +(15, 47) + + + +2923370542239283016->-6847290070637103654 + + + + + +-6847290070637103654->5532963792721227377 + + + + + +-6847271152609888271 + +(16, 47) + + + +-7015834146179027945->-6847271152609888271 + + + + + +-6847271152609888271->5532963792721227377 + + + + + +-6847272413883780328 + +(17, 47) + + + +2978920566541864118->-6847272413883780328 + + + + + +-6847272413883780328->5532963792721227377 + + + + + +-6847273675008546113 + +(18, 47) + + + +6977824086149921333->-6847273675008546113 + + + + + +-6847273675008546113->5532963792721227377 + + + + + +-6847274936282438170 + +(19, 47) + + + +-2275214144126059468->-6847274936282438170 + + + + + +-6847274936282438170->5532963792721227377 + + + + + +-6847276197407203955 + +(20, 47) + + + +2769876041474266851->-6847276197407203955 + + + + + +-6847276197407203955->5532963792721227377 + + + + + +-6847277458681096012 + +(21, 47) + + + +-5706879823683870590->-6847277458681096012 + + + + + +-6847277458681096012->5532963792721227377 + + + + + +-6847278719805861797 + +(22, 47) + + + +7960921375431606785->-6847278719805861797 + + + + + +-6847278719805861797->5532963792721227377 + + + + + +-6847279981042472286 + +(23, 47) + + + +2275291358589818704->-6847279981042472286 + + + + + +-6847279981042472286->5532963792721227377 + + + + + +-6847261063015256903 + +(24, 47) + + + +8343287195529033663->-6847261063015256903 + + + + + +-6847261063015256903->5532963792721227377 + + + + + +-6847262324289148960 + +(25, 47) + + + +-1407315231262688834->-6847262324289148960 + + + + + +-6847262324289148960->5532963792721227377 + + + + + +-6847263585413914745 + +(26, 47) + + + +5768157097700078141->-6847263585413914745 + + + + + +-6847263585413914745->5532963792721227377 + + + + + +-6847264846687806802 + +(27, 47) + + + +6183328958454558684->-6847264846687806802 + + + + + +-6847264846687806802->5532963792721227377 + + + + + +-6847266107812572587 + +(28, 47) + + + +-7223097358834314357->-6847266107812572587 + + + + + +-6847266107812572587->5532963792721227377 + + + + + +-6847267369086464644 + +(29, 47) + + + +4202557122767847818->-6847267369086464644 + + + + + +-6847267369086464644->5532963792721227377 + + + + + +-6847268630211230429 + +(30, 47) + + + +-4880515401238302103->-6847268630211230429 + + + + + +-6847268630211230429->5532963792721227377 + + + + + +-6847269891485122486 + +(31, 47) + + + +5325544214334185112->-6847269891485122486 + + + + + +-6847269891485122486->5532963792721227377 + + + + + +-6847250973532470239 + +(32, 47) + + + +-6043056782305386905->-6847250973532470239 + + + + + +-6847250973532470239->5532963792721227377 + + + + + +-6847252234657236024 + +(33, 47) + + + +752849344684607558->-6847252234657236024 + + + + + +-6847252234657236024->5532963792721227377 + + + + + +-6847253495931128081 + +(34, 47) + + + +3996699622759655813->-6847253495931128081 + + + + + +-6847253495931128081->5532963792721227377 + + + + + +-6847254757055893866 + +(35, 47) + + + +-5263990145182737916->-6847254757055893866 + + + + + +-6847254757055893866->5532963792721227377 + + + + + +-6847256018329785923 + +(36, 47) + + + +-9104476696104558029->-6847256018329785923 + + + + + +-6847256018329785923->5532963792721227377 + + + + + +-6847257279454551708 + +(37, 47) + + + +-7751730854631264238->-6847257279454551708 + + + + + +-6847257279454551708->5532963792721227377 + + + + + +-6847258540728443765 + +(38, 47) + + + +-2124938975929313007->-6847258540728443765 + + + + + +-6847258540728443765->5532963792721227377 + + + + + +-6847259801890491118 + +(39, 47) + + + +-4640417911081350496->-6847259801890491118 + + + + + +-6847259801890491118->5532963792721227377 + + + + + +-6847240883937838871 + +(40, 47) + + + +-1946028650983452721->-6847240883937838871 + + + + + +-6847240883937838871->5532963792721227377 + + + + + +-6847242145062604656 + +(41, 47) + + + +6061239370187931726->-6847242145062604656 + + + + + +-6847242145062604656->5532963792721227377 + + + + + +-6847243406336496713 + +(42, 47) + + + +5790059440734633357->-6847243406336496713 + + + + + +-6847243406336496713->5532963792721227377 + + + + + +-6847244667461262498 + +(43, 47) + + + +4196112219265108844->-6847244667461262498 + + + + + +-6847244667461262498->5532963792721227377 + + + + + +-6847245928735154555 + +(44, 47) + + + +520253163814674075->-6847245928735154555 + + + + + +-6847245928735154555->5532963792721227377 + + + + + +-6847247189859920340 + +(45, 47) + + + +-279138258149790694->-6847247189859920340 + + + + + +-6847247189859920340->5532963792721227377 + + + + + +-6847248451133812397 + +(46, 47) + + + +4251433999186967993->-6847248451133812397 + + + + + +-6847248451133812397->5532963792721227377 + + + + + +-6847249712258578182 + +(47, 47) + + + +-2227311892354026136->-6847249712258578182 + + + + + +-6847249712258578182->5532963792721227377 + + + + + +-6847230794380489071 + +(48, 47) + + + +-2946313899392806473->-6847230794380489071 + + + + + +-6847230794380489071->5532963792721227377 + + + + + +-6847291331777015076 + +(0, 48) + + + +5408852422873980650->-6847291331777015076 + + + + + +6633631137922614652 + +simple-shuffle-combine + + + +-6847291331777015076->6633631137922614652 + + + + + +-6847292593048577035 + +(1, 48) + + + +6021556619709705003->-6847292593048577035 + + + + + +-6847292593048577035->6633631137922614652 + + + + + +-6847293854171012722 + +(2, 48) + + + +-2247717051537387588->-6847293854171012722 + + + + + +-6847293854171012722->6633631137922614652 + + + + + +-6847295115442574681 + +(3, 48) + + + +-3008319126361884579->-6847295115442574681 + + + + + +-6847295115442574681->6633631137922614652 + + + + + +-6847296376565010368 + +(4, 48) + + + +7625060126592854046->-6847296376565010368 + + + + + +-6847296376565010368->6633631137922614652 + + + + + +-6847297637836572327 + +(5, 48) + + + +4791816337981140063->-6847297637836572327 + + + + + +-6847297637836572327->6633631137922614652 + + + + + +-6847298898959008014 + +(6, 48) + + + +-1314437711803948928->-6847298898959008014 + + + + + +-6847298898959008014->6633631137922614652 + + + + + +-6847300160118725269 + +(7, 48) + + + +3111368388735931377->-6847300160118725269 + + + + + +-6847300160118725269->6633631137922614652 + + + + + +-6847281242201024492 + +(8, 48) + + + +-6246766954595371038->-6847281242201024492 + + + + + +-6847281242201024492->6633631137922614652 + + + + + +-6847282503472586451 + +(9, 48) + + + +-672948230417510269->-6847282503472586451 + + + + + +-6847282503472586451->6633631137922614652 + + + + + +-6847283764595022138 + +(10, 48) + + + +6688660259925691924->-6847283764595022138 + + + + + +-6847283764595022138->6633631137922614652 + + + + + +-6847285025866584097 + +(11, 48) + + + +-3991005168221952427->-6847285025866584097 + + + + + +-6847285025866584097->6633631137922614652 + + + + + +-6847286286989019784 + +(12, 48) + + + +6713277375217317398->-6847286286989019784 + + + + + +-6847286286989019784->6633631137922614652 + + + + + +-6847287548260581743 + +(13, 48) + + + +4884914100384391607->-6847287548260581743 + + + + + +-6847287548260581743->6633631137922614652 + + + + + +-6847288809383017430 + +(14, 48) + + + +-3368405542318094984->-6847288809383017430 + + + + + +-6847288809383017430->6633631137922614652 + + + + + +-6847290070654579389 + +(15, 48) + + + +-4183048609707923575->-6847290070654579389 + + + + + +-6847290070654579389->6633631137922614652 + + + + + +-6847271152587752340 + +(16, 48) + + + +-6084355700711270310->-6847271152587752340 + + + + + +-6847271152587752340->6633631137922614652 + + + + + +-6847272413859314299 + +(17, 48) + + + +-8700906811781719141->-6847272413859314299 + + + + + +-6847272413859314299->6633631137922614652 + + + + + +-6847273674981749986 + +(18, 48) + + + +-8323223006732261076->-6847273674981749986 + + + + + +-6847273674981749986->6633631137922614652 + + + + + +-6847274936253311945 + +(19, 48) + + + +1188201927724267789->-6847274936253311945 + + + + + +-6847274936253311945->6633631137922614652 + + + + + +-6847276197375747632 + +(20, 48) + + + +-5136726695656620658->-6847276197375747632 + + + + + +-6847276197375747632->6633631137922614652 + + + + + +-6847277458647309591 + +(21, 48) + + + +-7913733734018333233->-6847277458647309591 + + + + + +-6847277458647309591->6633631137922614652 + + + + + +-6847278719769745278 + +(22, 48) + + + +1687289387074335792->-6847278719769745278 + + + + + +-6847278719769745278->6633631137922614652 + + + + + +-6847279981078588805 + +(23, 48) + + + +147483825029438753->-6847279981078588805 + + + + + +-6847279981078588805->6633631137922614652 + + + + + +-6847261063011761756 + +(24, 48) + + + +1910055769872256338->-6847261063011761756 + + + + + +-6847261063011761756->6633631137922614652 + + + + + +-6847262324283323715 + +(25, 48) + + + +-5700995622656904909->-6847262324283323715 + + + + + +-6847262324283323715->6633631137922614652 + + + + + +-6847263585405759402 + +(26, 48) + + + +3887445310551288516->-6847263585405759402 + + + + + +-6847263585405759402->6633631137922614652 + + + + + +-6847264846677321361 + +(27, 48) + + + +3698959137933080325->-6847264846677321361 + + + + + +-6847264846677321361->6633631137922614652 + + + + + +-6847266107799757048 + +(28, 48) + + + +-5229021741818739066->-6847266107799757048 + + + + + +-6847266107799757048->6633631137922614652 + + + + + +-6847267369071319007 + +(29, 48) + + + +3229557478287929959->-6847267369071319007 + + + + + +-6847267369071319007->6633631137922614652 + + + + + +-6847268630193754694 + +(30, 48) + + + +2141121775015387688->-6847268630193754694 + + + + + +-6847268630193754694->6633631137922614652 + + + + + +-6847269891465316653 + +(31, 48) + + + +-6865133564053739463->-6847269891465316653 + + + + + +-6847269891465316653->6633631137922614652 + + + + + +-6847250973547615876 + +(32, 48) + + + +-4955764415198192246->-6847250973547615876 + + + + + +-6847250973547615876->6633631137922614652 + + + + + +-6847252234670051563 + +(33, 48) + + + +8671389112616533067->-6847252234670051563 + + + + + +-6847252234670051563->6633631137922614652 + + + + + +-6847253495941613522 + +(34, 48) + + + +-2138528475209387428->-6847253495941613522 + + + + + +-6847253495941613522->6633631137922614652 + + + + + +-6847254757064049209 + +(35, 48) + + + +3827871021476048509->-6847254757064049209 + + + + + +-6847254757064049209->6633631137922614652 + + + + + +-6847256018335611168 + +(36, 48) + + + +-1532644585536453954->-6847256018335611168 + + + + + +-6847256018335611168->6633631137922614652 + + + + + +-6847257279458046855 + +(37, 48) + + + +-4873060480262858113->-6847257279458046855 + + + + + +-6847257279458046855->6633631137922614652 + + + + + +-6847258540729608814 + +(38, 48) + + + +38857142008807968->-6847258540729608814 + + + + + +-6847258540729608814->6633631137922614652 + + + + + +-6847259801889326069 + +(39, 48) + + + +-6804214030127977071->-6847259801889326069 + + + + + +-6847259801889326069->6633631137922614652 + + + + + +-6847240883971625292 + +(40, 48) + + + +199755535346145922->-6847240883971625292 + + + + + +-6847240883971625292->6633631137922614652 + + + + + +-6847242145094060979 + +(41, 48) + + + +-2024317778046866781->-6847242145094060979 + + + + + +-6847242145094060979->6633631137922614652 + + + + + +-6847243406365622938 + +(42, 48) + + + +2233399299769093300->-6847243406365622938 + + + + + +-6847243406365622938->6633631137922614652 + + + + + +-6847244667488058625 + +(43, 48) + + + +995700795413278837->-6847244667488058625 + + + + + +-6847244667488058625->6633631137922614652 + + + + + +-6847245928759620584 + +(44, 48) + + + +-4565176265464439882->-6847245928759620584 + + + + + +-6847245928759620584->6633631137922614652 + + + + + +-6847247189882056271 + +(45, 48) + + + +-2396436594116466729->-6847247189882056271 + + + + + +-6847247189882056271->6633631137922614652 + + + + + +-6847248451153618230 + +(46, 48) + + + +805308981296170008->-6847248451153618230 + + + + + +-6847248451153618230->6633631137922614652 + + + + + +-6847249712276053917 + +(47, 48) + + + +-2236560016615984727->-6847249712276053917 + + + + + +-6847249712276053917->6633631137922614652 + + + + + +-6847230794358353140 + +(48, 48) + + + +-9197922129803787782->-6847230794358353140 + + + + + +-6847230794358353140->6633631137922614652 + + + + + +-7271825275853727618 + +0 + + + +1491115096712983436->-7271825275853727618 + + + + + +-4911192465250854 + +set_index_post_scalar + + + +-7271825275853727618->-4911192465250854 + + + + + +-7271825275854810143 + +1 + + + +1237898002951376167->-7271825275854810143 + + + + + +-7313543989499648795 + +set_index_post_scalar + + + +-7271825275854810143->-7313543989499648795 + + + + + +-7271825275855892668 + +2 + + + +2268567409594649842->-7271825275855892668 + + + + + +-1509939344103742604 + +set_index_post_scalar + + + +-7271825275855892668->-1509939344103742604 + + + + + +-7271825275856975193 + +3 + + + +3369230225407514717->-7271825275856975193 + + + + + +-412651238404355729 + +set_index_post_scalar + + + +-7271825275856975193->-412651238404355729 + + + + + +-7271825275858057718 + +4 + + + +4539803893240303192->-7271825275858057718 + + + + + +4860752710654100246 + +set_index_post_scalar + + + +-7271825275858057718->4860752710654100246 + + + + + +-7271825275859140243 + +5 + + + +6082374622813686467->-7271825275859140243 + + + + + +-2599982868675828095 + +set_index_post_scalar + + + +-7271825275859140243->-2599982868675828095 + + + + + +-7271825275860222768 + +6 + + + +7134231313787865742->-7271825275860222768 + + + + + +2708497527824114496 + +set_index_post_scalar + + + +-7271825275860222768->2708497527824114496 + + + + + +-7271825275861305293 + +7 + + + +8234894118550315417->-7271825275861305293 + + + + + +3734773230702352971 + +set_index_post_scalar + + + +-7271825275861305293->3734773230702352971 + + + + + +-7271825275862387818 + +8 + + + +1493623177110034692->-7271825275862387818 + + + + + +8272988676127244962 + +set_index_post_scalar + + + +-7271825275862387818->8272988676127244962 + + + + + +-7271825275863470343 + +9 + + + +-5810866345050651649->-7271825275863470343 + + + + + +2010253609310366221 + +set_index_post_scalar + + + +-7271825275863470343->2010253609310366221 + + + + + +-7271825275864552868 + +10 + + + +-4710198997666893974->-7271825275864552868 + + + + + +6642855242033585212 + +set_index_post_scalar + + + +-7271825275864552868->6642855242033585212 + + + + + +-7271825275865635393 + +11 + + + +-3687284701743748299->-7271825275865635393 + + + + + +8859089597523515287 + +set_index_post_scalar + + + +-7271825275865635393->8859089597523515287 + + + + + +-7271825275866717918 + +12 + + + +6338118214775568592->-7271825275866717918 + + + + + +-4139016032248457154 + +set_index_post_scalar + + + +-7271825275866717918->-4139016032248457154 + + + + + +-7271825275867800443 + +13 + + + +-983835332281493349->-7271825275867800443 + + + + + +6836993384645722121 + +set_index_post_scalar + + + +-7271825275867800443->6836993384645722121 + + + + + +-7271825275868882968 + +14 + + + +38829656575316326->-7271825275868882968 + + + + + +-6411092019626099704 + +set_index_post_scalar + + + +-7271825275868882968->-6411092019626099704 + + + + + +-7271825275869965493 + +15 + + + +1139456137791624401->-7271825275869965493 + + + + + +-5239661011063071629 + +set_index_post_scalar + + + +-7271825275869965493->-5239661011063071629 + + + + + +-7271825275871048018 + +16 + + + +4462707218239603676->-7271825275871048018 + + + + + +327426172339157962 + +set_index_post_scalar + + + +-7271825275871048018->327426172339157962 + + + + + +-7271825275872130543 + +17 + + + +4373067005400823351->-7271825275872130543 + + + + + +-8324662077326083179 + +set_index_post_scalar + + + +-7271825275872130543->-8324662077326083179 + + + + + +-7271825275873213068 + +18 + + + +5395749863437186626->-7271825275873213068 + + + + + +5903628886674791012 + +set_index_post_scalar + + + +-7271825275873213068->5903628886674791012 + + + + + +-7271825275874295593 + +19 + + + +-3025610838906466899->-7271825275874295593 + + + + + +-2605431445772382113 + +set_index_post_scalar + + + +-7271825275874295593->-2605431445772382113 + + + + + +-7271825275875378118 + +20 + + + +-9196218965325156440->-7271825275875378118 + + + + + +5167455994239787462 + +set_index_post_scalar + + + +-7271825275875378118->5167455994239787462 + + + + + +-7271825275876460643 + +21 + + + +-8098914186541078765->-7271825275876460643 + + + + + +-3404708958399094479 + +set_index_post_scalar + + + +-7271825275876460643->-3404708958399094479 + + + + + +-7271825275877543168 + +22 + + + +-8114127481031040290->-7271825275877543168 + + + + + +2896487265427985712 + +set_index_post_scalar + + + +-7271825275877543168->2896487265427985712 + + + + + +-7271825275878625693 + +23 + + + +1915418191650649001->-7271825275878625693 + + + + + +3996881216775572987 + +set_index_post_scalar + + + +-7271825275878625693->3996881216775572987 + + + + + +-7271825275879708218 + +24 + + + +-4973298990768712940->-7271825275879708218 + + + + + +7270726111421552978 + +set_index_post_scalar + + + +-7271825275879708218->7270726111421552978 + + + + + +-7271825275880790743 + +25 + + + +-3802725357299598065->-7271825275880790743 + + + + + +-37903271450237763 + +set_index_post_scalar + + + +-7271825275880790743->-37903271450237763 + + + + + +-7271825275881873268 + +26 + + + +6222690932248597626->-7271825275881873268 + + + + + +-4320066951297568788 + +set_index_post_scalar + + + +-7271825275881873268->-4320066951297568788 + + + + + +-7271825275882955793 + +27 + + + +8426695571241120901->-7271825275882955793 + + + + + +-3219422041192981113 + +set_index_post_scalar + + + +-7271825275882955793->-3219422041192981113 + + + + + +-7271825275884038318 + +28 + + + +-76629382402763040->-7271825275884038318 + + + + + +-6331527419595352338 + +set_index_post_scalar + + + +-7271825275884038318->-6331527419595352338 + + + + + +-7271825275885120843 + +29 + + + +1024033431227731435->-7271825275885120843 + + + + + +5992407219418258937 + +set_index_post_scalar + + + +-7271825275885120843->5992407219418258937 + + + + + +-7271825275886203368 + +30 + + + +-7392883329842454090->-7271825275886203368 + + + + + +1058007585045622712 + +set_index_post_scalar + + + +-7271825275886203368->1058007585045622712 + + + + + +-7271825275887285893 + +31 + + + +-5175553430421902815->-7271825275887285893 + + + + + +-7432172328574114013 + +set_index_post_scalar + + + +-7271825275887285893->-7432172328574114013 + + + + + +-7271825275819086818 + +32 + + + +5702302575957607468->-7271825275819086818 + + + + + +5637365451053536314 + +set_index_post_scalar + + + +-7271825275819086818->5637365451053536314 + + + + + +-7271825275820169343 + +33 + + + +6873981168951948743->-7271825275820169343 + + + + + +6807993311594699589 + +set_index_post_scalar + + + +-7271825275820169343->6807993311594699589 + + + + + +-7271825275821251868 + +34 + + + +38999751266899602->-7271825275821251868 + + + + + +1359527727793054164 + +set_index_post_scalar + + + +-7271825275821251868->1359527727793054164 + + + + + +-7271825275822334393 + +35 + + + +-8321258247921784323->-7271825275822334393 + + + + + +2905381986139359439 + +set_index_post_scalar + + + +-7271825275822334393->2905381986139359439 + + + + + +-7271825275823416918 + +36 + + + +-7293895496873047048->-7271825275823416918 + + + + + +-7964888091013784202 + +set_index_post_scalar + + + +-7271825275823416918->-7964888091013784202 + + + + + +-7271825275824499443 + +37 + + + +-6197699014181410973->-7271825275824499443 + + + + + +-6794296507461527327 + +set_index_post_scalar + + + +-7271825275824499443->-6794296507461527327 + + + + + +-7271825275825581968 + +38 + + + +4940131213300269102->-7271825275825581968 + + + + + +7908601967954357664 + +set_index_post_scalar + + + +-7271825275825581968->7908601967954357664 + + + + + +-7271825275826664493 + +39 + + + +-3475659347025742023->-7271825275826664493 + + + + + +-581846835557003861 + +set_index_post_scalar + + + +-7271825275826664493->-581846835557003861 + + + + + +-7271825275827747018 + +40 + + + +-2379458326112179548->-7271825275827747018 + + + + + +-5744004759260626686 + +set_index_post_scalar + + + +-7271825275827747018->-5744004759260626686 + + + + + +-7271825275828829543 + +41 + + + +-1353203967595871073->-7271825275828829543 + + + + + +-4581132018342277011 + +set_index_post_scalar + + + +-7271825275828829543->-4581132018342277011 + + + + + +-7271825275829912068 + +42 + + + +9039872346297589002->-7271825275829912068 + + + + + +-7814365945731137636 + +set_index_post_scalar + + + +-7271825275829912068->-7814365945731137636 + + + + + +-7271825275830994593 + +43 + + + +692935105895691477->-7271825275830994593 + + + + + +-6095490608856689161 + +set_index_post_scalar + + + +-7271825275830994593->-6095490608856689161 + + + + + +-7271825275832077118 + +44 + + + +1789118194496842352->-7271825275832077118 + + + + + +-837303119772076386 + +set_index_post_scalar + + + +-7271825275832077118->-837303119772076386 + + + + + +-7271825275833159643 + +45 + + + +2815390449853383227->-7271825275833159643 + + + + + +263326775606838889 + +set_index_post_scalar + + + +-7271825275833159643->263326775606838889 + + + + + +-7271825275834242168 + +46 + + + +-4562113501990289914->-7271825275834242168 + + + + + +-3559607136380432536 + +set_index_post_scalar + + + +-7271825275834242168->-3559607136380432536 + + + + + +-7271825275835324693 + +47 + + + +5532963792721227377->-7271825275835324693 + + + + + +-1268731820329359661 + +set_index_post_scalar + + + +-7271825275835324693->-1268731820329359661 + + + + + +-7271825275836407218 + +48 + + + +6633631137922614652->-7271825275836407218 + + + + + +4753855166979404330 + +set_index_post_scalar + + + +-7271825275836407218->4753855166979404330 + + + + + +7026722324074434284 + +0 + + + +-4911192465250854->7026722324074434284 + + + + + +-7014193590482654006 + +sort_index + + + +7026722324074434284->-7014193590482654006 + + + + + +7026722324073351759 + +1 + + + +-7313543989499648795->7026722324073351759 + + + + + +-5987954254416916331 + +sort_index + + + +7026722324073351759->-5987954254416916331 + + + + + +7026722324076599334 + +2 + + + +-1509939344103742604->7026722324076599334 + + + + + +9155743814366524260 + +sort_index + + + +7026722324076599334->9155743814366524260 + + + + + +7026722324075516809 + +3 + + + +-412651238404355729->7026722324075516809 + + + + + +-8111534726033400481 + +sort_index + + + +7026722324075516809->-8111534726033400481 + + + + + +7026722324070104184 + +4 + + + +4860752710654100246->7026722324070104184 + + + + + +-504042411670997306 + +sort_index + + + +7026722324070104184->-504042411670997306 + + + + + +7026722324069021659 + +5 + + + +-2599982868675828095->7026722324069021659 + + + + + +-8851012563569518031 + +sort_index + + + +7026722324069021659->-8851012563569518031 + + + + + +7026722324072269234 + +6 + + + +2708497527824114496->7026722324072269234 + + + + + +-4385139311904469456 + +sort_index + + + +7026722324072269234->-4385139311904469456 + + + + + +7026722324071186709 + +7 + + + +3734773230702352971->7026722324071186709 + + + + + +-3283668757748050181 + +sort_index + + + +7026722324071186709->-3283668757748050181 + + + + + +7026722324083094484 + +8 + + + +8272988676127244962->7026722324083094484 + + + + + +1142352951164195410 + +sort_index + + + +7026722324083094484->1142352951164195410 + + + + + +7026722324082011959 + +9 + + + +2010253609310366221->7026722324082011959 + + + + + +2242978299487780285 + +sort_index + + + +7026722324082011959->2242978299487780285 + + + + + +7026722324085259534 + +10 + + + +6642855242033585212->7026722324085259534 + + + + + +-2620314940626814740 + +sort_index + + + +7026722324085259534->-2620314940626814740 + + + + + +7026722324084177009 + +11 + + + +8859089597523515287->7026722324084177009 + + + + + +37255560950182535 + +sort_index + + + +7026722324084177009->37255560950182535 + + + + + +7026722324078764384 + +12 + + + +-4139016032248457154->7026722324078764384 + + + + + +6591359208960867310 + +sort_index + + + +7026722324078764384->6591359208960867310 + + + + + +7026722324077681859 + +13 + + + +6836993384645722121->7026722324077681859 + + + + + +7614314300176385785 + +sort_index + + + +7026722324077681859->7614314300176385785 + + + + + +7026722324080929434 + +14 + + + +-6411092019626099704->7026722324080929434 + + + + + +2224143781286757560 + +sort_index + + + +7026722324080929434->2224143781286757560 + + + + + +7026722324079846909 + +15 + + + +-5239661011063071629->7026722324079846909 + + + + + +4978302426288537635 + +sort_index + + + +7026722324079846909->4978302426288537635 + + + + + +7026722324091754684 + +16 + + + +327426172339157962->7026722324091754684 + + + + + +-6240542526524925222 + +sort_index + + + +7026722324091754684->-6240542526524925222 + + + + + +7026722324090672159 + +17 + + + +-8324662077326083179->7026722324090672159 + + + + + +-5065484627552888347 + +sort_index + + + +7026722324090672159->-5065484627552888347 + + + + + +7026722324093919734 + +18 + + + +5903628886674791012->7026722324093919734 + + + + + +-7164694460199999372 + +sort_index + + + +7026722324093919734->-7164694460199999372 + + + + + +7026722324092837209 + +19 + + + +-2605431445772382113->7026722324092837209 + + + + + +-7328725179109047697 + +sort_index + + + +7026722324092837209->-7328725179109047697 + + + + + +7026722324087424584 + +20 + + + +5167455994239787462->7026722324087424584 + + + + + +436143952502265878 + +sort_index + + + +7026722324087424584->436143952502265878 + + + + + +7026722324086342059 + +21 + + + +-3404708958399094479->7026722324086342059 + + + + + +-7989645623232875647 + +sort_index + + + +7026722324086342059->-7989645623232875647 + + + + + +7026722324089589634 + +22 + + + +2896487265427985712->7026722324089589634 + + + + + +-2412344092423182272 + +sort_index + + + +7026722324089589634->-2412344092423182272 + + + + + +7026722324088507109 + +23 + + + +3996881216775572987->7026722324088507109 + + + + + +-1860019312395196597 + +sort_index + + + +7026722324088507109->-1860019312395196597 + + + + + +7026722324100414884 + +24 + + + +7270726111421552978->7026722324100414884 + + + + + +4340433037678680994 + +sort_index + + + +7026722324100414884->4340433037678680994 + + + + + +7026722324099332359 + +25 + + + +-37903271450237763->7026722324099332359 + + + + + +-4129683001067942131 + +sort_index + + + +7026722324099332359->-4129683001067942131 + + + + + +7026722324102579934 + +26 + + + +-4320066951297568788->7026722324102579934 + + + + + +2217141567622883644 + +sort_index + + + +7026722324102579934->2217141567622883644 + + + + + +7026722324101497409 + +27 + + + +-3219422041192981113->7026722324101497409 + + + + + +3239807689320455319 + +sort_index + + + +7026722324101497409->3239807689320455319 + + + + + +7026722324096084784 + +28 + + + +-6331527419595352338->7026722324096084784 + + + + + +7373905534925874494 + +sort_index + + + +7026722324096084784->7373905534925874494 + + + + + +7026722324095002259 + +29 + + + +5992407219418258937->7026722324095002259 + + + + + +8554496195894293769 + +sort_index + + + +7026722324095002259->8554496195894293769 + + + + + +7026722324098249834 + +30 + + + +1058007585045622712->7026722324098249834 + + + + + +6935128220784993544 + +sort_index + + + +7026722324098249834->6935128220784993544 + + + + + +7026722324097167309 + +31 + + + +-7432172328574114013->7026722324097167309 + + + + + +6273260624838607219 + +sort_index + + + +7026722324097167309->6273260624838607219 + + + + + +7026722324039793484 + +32 + + + +5637365451053536314->7026722324039793484 + + + + + +7146131779240558378 + +sort_index + + + +7026722324039793484->7146131779240558378 + + + + + +7026722324038710959 + +33 + + + +6807993311594699589->7026722324038710959 + + + + + +8659328058762757109 + +sort_index + + + +7026722324038710959->8659328058762757109 + + + + + +7026722324041958534 + +34 + + + +1359527727793054164->7026722324041958534 + + + + + +4841091911219065284 + +sort_index + + + +7026722324041958534->4841091911219065284 + + + + + +7026722324040876009 + +35 + + + +2905381986139359439->7026722324040876009 + + + + + +5975503951989203903 + +sort_index + + + +7026722324040876009->5975503951989203903 + + + + + +7026722324035463384 + +36 + + + +-7964888091013784202->7026722324035463384 + + + + + +1813203574556196134 + +sort_index + + + +7026722324035463384->1813203574556196134 + + + + + +7026722324034380859 + +37 + + + +-6794296507461527327->7026722324034380859 + + + + + +-5618571739503921007 + +sort_index + + + +7026722324034380859->-5618571739503921007 + + + + + +7026722324037628434 + +38 + + + +7908601967954357664->7026722324037628434 + + + + + +-8686807366277394032 + +sort_index + + + +7026722324037628434->-8686807366277394032 + + + + + +7026722324036545909 + +39 + + + +-581846835557003861->7026722324036545909 + + + + + +711486596556506459 + +sort_index + + + +7026722324036545909->711486596556506459 + + + + + +7026722324048453684 + +40 + + + +-5744004759260626686->7026722324048453684 + + + + + +4033998581873664178 + +sort_index + + + +7026722324048453684->4033998581873664178 + + + + + +7026722324047371159 + +41 + + + +-4581132018342277011->7026722324047371159 + + + + + +-3127419372266155747 + +sort_index + + + +7026722324047371159->-3127419372266155747 + + + + + +7026722324050618734 + +42 + + + +-7814365945731137636->7026722324050618734 + + + + + +1809435732058538828 + +sort_index + + + +7026722324050618734->1809435732058538828 + + + + + +7026722324049536209 + +43 + + + +-6095490608856689161->7026722324049536209 + + + + + +2933368693145782503 + +sort_index + + + +7026722324049536209->2933368693145782503 + + + + + +7026722324044123584 + +44 + + + +-837303119772076386->7026722324044123584 + + + + + +2244979112534284878 + +sort_index + + + +7026722324044123584->2244979112534284878 + + + + + +7026722324043041059 + +45 + + + +263326775606838889->7026722324043041059 + + + + + +3828140412432299609 + +sort_index + + + +7026722324043041059->3828140412432299609 + + + + + +7026722324046288634 + +46 + + + +-3559607136380432536->7026722324046288634 + + + + + +6751573059833922328 + +sort_index + + + +7026722324046288634->6751573059833922328 + + + + + +7026722324045206109 + +47 + + + +-1268731820329359661->7026722324045206109 + + + + + +1148763032606787203 + +sort_index + + + +7026722324045206109->1148763032606787203 + + + + + +7026722324057113884 + +48 + + + +4753855166979404330->7026722324057113884 + + + + + +-1543869725835498182 + +sort_index + + + +7026722324057113884->-1543869725835498182 + + + + + +4765995075540723644 + +0 + + + +-7014193590482654006->4765995075540723644 + + + + + +-5546164580954905022 + +head-partial + + + +4765995075540723644->-5546164580954905022 + + + + + +4765995075539641119 + +1 + + + +-5987954254416916331->4765995075539641119 + + + + + +-3180863524948427347 + +head-partial + + + +4765995075539641119->-3180863524948427347 + + + + + +4765995075542888694 + +2 + + + +9155743814366524260->4765995075542888694 + + + + + +1852566330927576028 + +head-partial + + + +4765995075542888694->1852566330927576028 + + + + + +4765995075541806169 + +3 + + + +-8111534726033400481->4765995075541806169 + + + + + +2878784391393371703 + +head-partial + + + +4765995075541806169->2878784391393371703 + + + + + +4765995075536393544 + +4 + + + +-504042411670997306->4765995075536393544 + + + + + +-701406277903199522 + +head-partial + + + +4765995075536393544->-701406277903199522 + + + + + +4765995075535311019 + +5 + + + +-8851012563569518031->4765995075535311019 + + + + + +1047286724830602153 + +head-partial + + + +4765995075535311019->1047286724830602153 + + + + + +4765995075538558594 + +6 + + + +-4385139311904469456->4765995075538558594 + + + + + +6697020539006345128 + +head-partial + + + +4765995075538558594->6697020539006345128 + + + + + +4765995075537476069 + +7 + + + +-3283668757748050181->4765995075537476069 + + + + + +7719975701131581203 + +head-partial + + + +4765995075537476069->7719975701131581203 + + + + + +4765995075549383844 + +8 + + + +1142352951164195410->4765995075549383844 + + + + + +4712580560937747578 + +head-partial + + + +4765995075549383844->4712580560937747578 + + + + + +4765995075548301319 + +9 + + + +2242978299487780285->4765995075548301319 + + + + + +5892871676326814853 + +head-partial + + + +4765995075548301319->5892871676326814853 + + + + + +4765995075551548894 + +10 + + + +-2620314940626814740->4765995075551548894 + + + + + +-6334628258972286188 + +head-partial + + + +4765995075551548894->-6334628258972286188 + + + + + +4765995075550466369 + +11 + + + +37255560950182535->4765995075550466369 + + + + + +-5312834593500713713 + +head-partial + + + +4765995075550466369->-5312834593500713713 + + + + + +4765995075545053744 + +12 + + + +6591359208960867310->4765995075545053744 + + + + + +8995445263787069878 + +head-partial + + + +4765995075545053744->8995445263787069878 + + + + + +4765995075543971219 + +13 + + + +7614314300176385785->4765995075543971219 + + + + + +-8424218870469879263 + +head-partial + + + +4765995075543971219->-8424218870469879263 + + + + + +4765995075547218794 + +14 + + + +2224143781286757560->4765995075547218794 + + + + + +-1494598444813875488 + +head-partial + + + +4765995075547218794->-1494598444813875488 + + + + + +4765995075546136269 + +15 + + + +4978302426288537635->4765995075546136269 + + + + + +-468322777407816213 + +head-partial + + + +4765995075546136269->-468322777407816213 + + + + + +4765995075523403244 + +16 + + + +-6240542526524925222->4765995075523403244 + + + + + +1967989107310001394 + +head-partial + + + +4765995075523403244->1967989107310001394 + + + + + +4765995075522320719 + +17 + + + +-5065484627552888347->4765995075522320719 + + + + + +3143009556494207069 + +head-partial + + + +4765995075522320719->3143009556494207069 + + + + + +4765995075525568294 + +18 + + + +-7164694460199999372->4765995075525568294 + + + + + +8341399363700990860 + +head-partial + + + +4765995075525568294->8341399363700990860 + + + + + +4765995075524485769 + +19 + + + +-7328725179109047697->4765995075524485769 + + + + + +955037610774887719 + +head-partial + + + +4765995075524485769->955037610774887719 + + + + + +4765995075519073144 + +20 + + + +436143952502265878->4765995075519073144 + + + + + +6812447866930084494 + +head-partial + + + +4765995075519073144->6812447866930084494 + + + + + +4765995075517990619 + +21 + + + +-7989645623232875647->4765995075517990619 + + + + + +-1096469258425200231 + +head-partial + + + +4765995075517990619->-1096469258425200231 + + + + + +4765995075521238194 + +22 + + + +-2412344092423182272->4765995075521238194 + + + + + +-5282765043382239656 + +head-partial + + + +4765995075521238194->-5282765043382239656 + + + + + +4765995075520155669 + +23 + + + +-1860019312395196597->4765995075520155669 + + + + + +5707409209776782019 + +head-partial + + + +4765995075520155669->5707409209776782019 + + + + + +4765995075532063444 + +24 + + + +4340433037678680994->4765995075532063444 + + + + + +-6185232420798390422 + +head-partial + + + +4765995075532063444->-6185232420798390422 + + + + + +4765995075530980919 + +25 + + + +-4129683001067942131->4765995075530980919 + + + + + +3762436485078326069 + +head-partial + + + +4765995075530980919->3762436485078326069 + + + + + +4765995075534228494 + +26 + + + +2217141567622883644->4765995075534228494 + + + + + +-7218173800175531772 + +head-partial + + + +4765995075534228494->-7218173800175531772 + + + + + +4765995075533145969 + +27 + + + +3239807689320455319->4765995075533145969 + + + + + +-6095649808214199297 + +head-partial + + + +4765995075533145969->-6095649808214199297 + + + + + +4765995075527733344 + +28 + + + +7373905534925874494->4765995075527733344 + + + + + +8770903589478465894 + +head-partial + + + +4765995075527733344->8770903589478465894 + + + + + +4765995075526650819 + +29 + + + +8554496195894293769->4765995075526650819 + + + + + +-8082011661192313647 + +head-partial + + + +4765995075526650819->-8082011661192313647 + + + + + +4765995075529898394 + +30 + + + +6935128220784993544->4765995075529898394 + + + + + +-2430355908922234672 + +head-partial + + + +4765995075529898394->-2430355908922234672 + + + + + +4765995075528815869 + +31 + + + +6273260624838607219->4765995075528815869 + + + + + +-1250869102264317797 + +head-partial + + + +4765995075528815869->-1250869102264317797 + + + + + +4765995075506082844 + +32 + + + +7146131779240558378->4765995075506082844 + + + + + +1198766874620547810 + +head-partial + + + +4765995075506082844->1198766874620547810 + + + + + +4765995075505000319 + +33 + + + +8659328058762757109->4765995075505000319 + + + + + +2294700141826084685 + +head-partial + + + +4765995075505000319->2294700141826084685 + + + + + +4765995075508247894 + +34 + + + +4841091911219065284->4765995075508247894 + + + + + +-2713784831718967940 + +head-partial + + + +4765995075508247894->-2713784831718967940 + + + + + +4765995075507165369 + +35 + + + +5975503951989203903->4765995075507165369 + + + + + +23713516087448535 + +head-partial + + + +4765995075507165369->23713516087448535 + + + + + +4765995075501752744 + +36 + + + +1813203574556196134->4765995075501752744 + + + + + +6021041398770781310 + +head-partial + + + +4765995075501752744->6021041398770781310 + + + + + +4765995075500670219 + +37 + + + +-5618571739503921007->4765995075500670219 + + + + + +7121708743902886985 + +head-partial + + + +4765995075500670219->7121708743902886985 + + + + + +4765995075503917794 + +38 + + + +-8686807366277394032->4765995075503917794 + + + + + +2118488772081759560 + +head-partial + + + +4765995075503917794->2118488772081759560 + + + + + +4765995075502835269 + +39 + + + +711486596556506459->4765995075502835269 + + + + + +4924862812936698035 + +head-partial + + + +4765995075502835269->4924862812936698035 + + + + + +4765995075514743044 + +40 + + + +4033998581873664178->4765995075514743044 + + + + + +9033362935198570010 + +head-partial + + + +4765995075514743044->9033362935198570010 + + + + + +4765995075513660519 + +41 + + + +-3127419372266155747->4765995075513660519 + + + + + +-8360398253443661531 + +head-partial + + + +4765995075513660519->-8360398253443661531 + + + + + +4765995075516908094 + +42 + + + +1809435732058538828->4765995075516908094 + + + + + +6469803914089364660 + +head-partial + + + +4765995075516908094->6469803914089364660 + + + + + +4765995075515825569 + +43 + + + +2933368693145782503->4765995075515825569 + + + + + +7937125660609359535 + +head-partial + + + +4765995075515825569->7937125660609359535 + + + + + +4765995075510412944 + +44 + + + +2244979112534284878->4765995075510412944 + + + + + +-4546608841750696106 + +head-partial + + + +4765995075510412944->-4546608841750696106 + + + + + +4765995075509330419 + +45 + + + +3828140412432299609->4765995075509330419 + + + + + +-1758106866443687231 + +head-partial + + + +4765995075509330419->-1758106866443687231 + + + + + +4765995075512577994 + +46 + + + +6751573059833922328->4765995075512577994 + + + + + +-7260057359660073856 + +head-partial + + + +4765995075512577994->-7260057359660073856 + + + + + +4765995075511495469 + +47 + + + +1148763032606787203->4765995075511495469 + + + + + +-5648362178734065781 + +head-partial + + + +4765995075511495469->-5648362178734065781 + + + + + +4765995075488762444 + +48 + + + +-1543869725835498182->4765995075488762444 + + + + + +-2567685613708360046 + +head-partial + + + +4765995075488762444->-2567685613708360046 + + + + + +5832550734064391540 + +0 + + + +-5546164580954905022->5832550734064391540 + + + + + +1220865840013551495 + +head + + + +5832550734064391540->1220865840013551495 + + + + + +5832550734063309015 + +1 + + + +-3180863524948427347->5832550734063309015 + + + + + +5832550734063309015->1220865840013551495 + + + + + +5832550734066556590 + +2 + + + +1852566330927576028->5832550734066556590 + + + + + +5832550734066556590->1220865840013551495 + + + + + +5832550734065474065 + +3 + + + +2878784391393371703->5832550734065474065 + + + + + +5832550734065474065->1220865840013551495 + + + + + +5832550734060061440 + +4 + + + +-701406277903199522->5832550734060061440 + + + + + +5832550734060061440->1220865840013551495 + + + + + +5832550734058978915 + +5 + + + +1047286724830602153->5832550734058978915 + + + + + +5832550734058978915->1220865840013551495 + + + + + +5832550734062226490 + +6 + + + +6697020539006345128->5832550734062226490 + + + + + +5832550734062226490->1220865840013551495 + + + + + +5832550734061143965 + +7 + + + +7719975701131581203->5832550734061143965 + + + + + +5832550734061143965->1220865840013551495 + + + + + +5832550734055731340 + +8 + + + +4712580560937747578->5832550734055731340 + + + + + +5832550734055731340->1220865840013551495 + + + + + +5832550734054648815 + +9 + + + +5892871676326814853->5832550734054648815 + + + + + +5832550734054648815->1220865840013551495 + + + + + +5832550734057896390 + +10 + + + +-6334628258972286188->5832550734057896390 + + + + + +5832550734057896390->1220865840013551495 + + + + + +5832550734056813865 + +11 + + + +-5312834593500713713->5832550734056813865 + + + + + +5832550734056813865->1220865840013551495 + + + + + +5832550734051401240 + +12 + + + +8995445263787069878->5832550734051401240 + + + + + +5832550734051401240->1220865840013551495 + + + + + +5832550734050318715 + +13 + + + +-8424218870469879263->5832550734050318715 + + + + + +5832550734050318715->1220865840013551495 + + + + + +5832550734053566290 + +14 + + + +-1494598444813875488->5832550734053566290 + + + + + +5832550734053566290->1220865840013551495 + + + + + +5832550734052483765 + +15 + + + +-468322777407816213->5832550734052483765 + + + + + +5832550734052483765->1220865840013551495 + + + + + +5832550734081711940 + +16 + + + +1967989107310001394->5832550734081711940 + + + + + +5832550734081711940->1220865840013551495 + + + + + +5832550734080629415 + +17 + + + +3143009556494207069->5832550734080629415 + + + + + +5832550734080629415->1220865840013551495 + + + + + +5832550734083876990 + +18 + + + +8341399363700990860->5832550734083876990 + + + + + +5832550734083876990->1220865840013551495 + + + + + +5832550734082794465 + +19 + + + +955037610774887719->5832550734082794465 + + + + + +5832550734082794465->1220865840013551495 + + + + + +5832550734077381840 + +20 + + + +6812447866930084494->5832550734077381840 + + + + + +5832550734077381840->1220865840013551495 + + + + + +5832550734076299315 + +21 + + + +-1096469258425200231->5832550734076299315 + + + + + +5832550734076299315->1220865840013551495 + + + + + +5832550734079546890 + +22 + + + +-5282765043382239656->5832550734079546890 + + + + + +5832550734079546890->1220865840013551495 + + + + + +5832550734078464365 + +23 + + + +5707409209776782019->5832550734078464365 + + + + + +5832550734078464365->1220865840013551495 + + + + + +5832550734073051740 + +24 + + + +-6185232420798390422->5832550734073051740 + + + + + +5832550734073051740->1220865840013551495 + + + + + +5832550734071969215 + +25 + + + +3762436485078326069->5832550734071969215 + + + + + +5832550734071969215->1220865840013551495 + + + + + +5832550734075216790 + +26 + + + +-7218173800175531772->5832550734075216790 + + + + + +5832550734075216790->1220865840013551495 + + + + + +5832550734074134265 + +27 + + + +-6095649808214199297->5832550734074134265 + + + + + +5832550734074134265->1220865840013551495 + + + + + +5832550734068721640 + +28 + + + +8770903589478465894->5832550734068721640 + + + + + +5832550734068721640->1220865840013551495 + + + + + +5832550734067639115 + +29 + + + +-8082011661192313647->5832550734067639115 + + + + + +5832550734067639115->1220865840013551495 + + + + + +5832550734070886690 + +30 + + + +-2430355908922234672->5832550734070886690 + + + + + +5832550734070886690->1220865840013551495 + + + + + +5832550734069804165 + +31 + + + +-1250869102264317797->5832550734069804165 + + + + + +5832550734069804165->1220865840013551495 + + + + + +5832550734099032340 + +32 + + + +1198766874620547810->5832550734099032340 + + + + + +5832550734099032340->1220865840013551495 + + + + + +5832550734097949815 + +33 + + + +2294700141826084685->5832550734097949815 + + + + + +5832550734097949815->1220865840013551495 + + + + + +5832550734101197390 + +34 + + + +-2713784831718967940->5832550734101197390 + + + + + +5832550734101197390->1220865840013551495 + + + + + +5832550734100114865 + +35 + + + +23713516087448535->5832550734100114865 + + + + + +5832550734100114865->1220865840013551495 + + + + + +5832550734094702240 + +36 + + + +6021041398770781310->5832550734094702240 + + + + + +5832550734094702240->1220865840013551495 + + + + + +5832550734093619715 + +37 + + + +7121708743902886985->5832550734093619715 + + + + + +5832550734093619715->1220865840013551495 + + + + + +5832550734096867290 + +38 + + + +2118488772081759560->5832550734096867290 + + + + + +5832550734096867290->1220865840013551495 + + + + + +5832550734095784765 + +39 + + + +4924862812936698035->5832550734095784765 + + + + + +5832550734095784765->1220865840013551495 + + + + + +5832550734090372140 + +40 + + + +9033362935198570010->5832550734090372140 + + + + + +5832550734090372140->1220865840013551495 + + + + + +5832550734089289615 + +41 + + + +-8360398253443661531->5832550734089289615 + + + + + +5832550734089289615->1220865840013551495 + + + + + +5832550734092537190 + +42 + + + +6469803914089364660->5832550734092537190 + + + + + +5832550734092537190->1220865840013551495 + + + + + +5832550734091454665 + +43 + + + +7937125660609359535->5832550734091454665 + + + + + +5832550734091454665->1220865840013551495 + + + + + +5832550734086042040 + +44 + + + +-4546608841750696106->5832550734086042040 + + + + + +5832550734086042040->1220865840013551495 + + + + + +5832550734084959515 + +45 + + + +-1758106866443687231->5832550734084959515 + + + + + +5832550734084959515->1220865840013551495 + + + + + +5832550734088207090 + +46 + + + +-7260057359660073856->5832550734088207090 + + + + + +5832550734088207090->1220865840013551495 + + + + + +5832550734087124565 + +47 + + + +-5648362178734065781->5832550734087124565 + + + + + +5832550734087124565->1220865840013551495 + + + + + +5832550734116352740 + +48 + + + +-2567685613708360046->5832550734116352740 + + + + + +5832550734116352740->1220865840013551495 + + + + + +-728789516192214975 + +0 + + + +1220865840013551495->-728789516192214975 + + + + + diff --git a/release/raydp_tests/dask_on_ray/a.txt b/release/raydp_tests/dask_on_ray/a.txt new file mode 100644 index 000000000000..ecda41c4c39c --- /dev/null +++ b/release/raydp_tests/dask_on_ray/a.txt @@ -0,0 +1,6 @@ +Clearing old data from ~/dask-on-ray-data. +Using task-based Dask shuffle +Trial 0 start +Trial 0 done after 26.13987922668457 +num obj ref bytes used: 474.84737396240234 +ray mean over 1 trials: 26.13987922668457 +- 0.0 diff --git a/release/raydp_tests/dask_on_ray/analyze.py b/release/raydp_tests/dask_on_ray/analyze.py new file mode 100644 index 000000000000..e3c71b88a30f --- /dev/null +++ b/release/raydp_tests/dask_on_ray/analyze.py @@ -0,0 +1,60 @@ + +raylet = "/tmp/ray/session_latest/logs/raylet.err" +result = "result.txt" + +from collections import defaultdict + +def analyze(filename): + object_refs = defaultdict(int) + object_refs_referenced = defaultdict(int) + object_refs_funcs = defaultdict(set) + func_cnt = defaultdict(int) + total_sizes = 0 + total_sizes_referenced_more_than_once = 0 + with open(filename) as f: + for line in f.readlines(): + if "[GET]" in line: + tokens = line.strip("\n").split("[GET]")[1].split(" ") + object_id = tokens[1].split(":")[1] + size = tokens[2].split(":")[1] + if len(tokens) >= 4: + func = tokens[3].split(":")[1] + object_refs_funcs[object_id].add(func) + func_cnt[func] += 1 + object_refs[object_id] += int(size) + object_refs_referenced[object_id] += 1 + total_sizes += int(size) + if object_refs[object_id] > 1: + total_sizes_referenced_more_than_once += int(size) + # print(filename) + print(f"total object refs: {len(object_refs)}") + # for object_id in object_refs.keys(): + # cnt = object_refs_referenced[object_id] + # size = object_refs[object_id] + # if cnt > 1: + # print(f"{object_id} uses {int(size) /1024 /1024}MB, referenced: {cnt}") + # print(f"functions: {object_refs_funcs[object_id]}") + # print(f"total sizes: {total_sizes / 1024 / 1024} MB") + # print(f"total sizes referenced more than once: {total_sizes_referenced_more_than_once / 1024 / 1024} MB") + return object_refs, object_refs_referenced, object_refs_funcs, func_cnt + +sizes_dict, _, _, _ = analyze(raylet) +_, ref_count, funcs, func_cnt = analyze(result) + +c = 0 +for object_id in ref_count.keys(): + cnt = int(ref_count[object_id]) + size = sizes_dict.get(object_id, None) + if not size: + continue + + if int(size) > 100 * 1024: + c += 1 + print(f"{object_id} uses {int(size) /1024 /1024}MB, referenced: {cnt}") + print(f"functions: {funcs[object_id]}") +assert c == len(sizes_dict), len(sizes_dict) +from pprint import pprint +pprint(func_cnt) +# print(f"total sizes: {total_sizes / 1024 / 1024} MB") +# print(f"total sizes referenced more than once: {total_sizes_referenced_more_than_once / 1024 / 1024} MB") + diff --git a/release/raydp_tests/dask_on_ray/output.csv b/release/raydp_tests/dask_on_ray/output.csv new file mode 100644 index 000000000000..baa6dcdd8936 --- /dev/null +++ b/release/raydp_tests/dask_on_ray/output.csv @@ -0,0 +1,45 @@ +system,operation,num_nodes,nbytes,npartitions,duration +ray,sort,1,1000000000,50,49.81460404396057 +ray,sort,1,100000000,50,26.4190411567688 +ray,sort,1,100000000,50,15.81236982345581 +ray,sort,1,100000000,50,37.54668378829956 +ray,sort,1,100000000,50,45.88498592376709 +ray,sort,1,100000000,2,6.961517095565796 +ray,sort,1,100000000,50,20.324124813079834 +ray,sort,1,100000000,50,39.60013222694397 +ray,sort,1,100000000,50,19.749282121658325 +ray,sort,1,100000000,50,14.275277853012085 +ray,sort,1,100000000,50,21.64378595352173 +ray,sort,1,100000000,50,6.137988805770874 +ray,sort,1,100000000,50,7.724100112915039 +ray,sort,1,100000000,50,16.97205114364624 +ray,sort,1,100000000,50,5.901505947113037 +ray,sort,1,100000000,50,7.561151027679443 +ray,sort,1,100000000,50,6.777873992919922 +ray,sort,1,100000000,50,22.736148834228516 +ray,sort,1,100000000,50,23.90089201927185 +ray,sort,1,100000000,50,47.67734980583191 +ray,sort,1,100000000,50,21.425940990447998 +ray,sort,1,100000000,50,28.597450971603394 +ray,sort,1,100000000,50,23.31282091140747 +ray,sort,1,100000000,50,18.240912914276123 +ray,sort,1,100000000,50,34.761038064956665 +ray,sort,1,100000000,50,23.420669317245483 +ray,sort,1,100000000,50,30.70834708213806 +ray,sort,1,100000000,50,25.204899072647095 +ray,sort,1,100000000,50,26.13987922668457 +ray,sort,1,100000000,50,24.668132066726685 +ray,sort,1,100000000,50,21.150716066360474 +ray,sort,1,100000000,50,24.247713088989258 +ray,sort,1,100000000,50,21.209686040878296 +ray,sort,1,100000000,50,18.561686038970947 +ray,sort,1,100000000,50,23.43537187576294 +ray,sort,1,100000000,50,17.797905206680298 +ray,sort,1,100000000,50,25.986549854278564 +ray,sort,1,100000000,50,18.29687786102295 +ray,sort,1,100000000,50,18.578434944152832 +ray,sort,1,100000000,50,17.5535569190979 +ray,sort,1,100000000,50,17.945733308792114 +ray,sort,1,100000000,50,17.20743417739868 +ray,sort,1,1000000000,20,41.81386184692383 +ray,sort,1,1000000000,20,76.37009787559509 diff --git a/release/raydp_tests/dask_on_ray/result.txt b/release/raydp_tests/dask_on_ray/result.txt new file mode 100644 index 000000000000..e855de52c305 --- /dev/null +++ b/release/raydp_tests/dask_on_ray/result.txt @@ -0,0 +1,1248 @@ +Clearing old data from ~/dask-on-ray-data. +Using task-based Dask shuffle +Trial 0 start +[GET] object_id:34c9c2094e42fdbfffffffffffffffffffffffff0100000001000000 size:0 func:max-540008a5-f6fe-46e9-aec7-4b822bab42fb +[GET] object_id:34c9c2094e42fdbfffffffffffffffffffffffff0100000001000000 size:0 func:min-fcd63f92-28e4-4a6e-89f3-5dfd4162e36d +[GET] object_id:34c9c2094e42fdbfffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-1-76560f47b9510925df1e3e2bf421692e', 0) +[GET] object_id:34c9c2094e42fdbfffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-0-76560f47b9510925df1e3e2bf421692e', 0) +[GET] object_id:44ed5e1383be6308ffffffffffffffffffffffff0100000001000000 size:0 func:max-cdfd1d82-8033-4557-8c42-bd249b8335d6 +[GET] object_id:44ed5e1383be6308ffffffffffffffffffffffff0100000001000000 size:0 func:min-de4a05be-2d3a-44ef-808e-b6d22f39be08 +[GET] object_id:44ed5e1383be6308ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-1-76560f47b9510925df1e3e2bf421692e', 1) +[GET] object_id:4e2ab276f14c37c2ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 0, 0) +[GET] object_id:139e431dd460af76ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 0, 0) +[GET] object_id:acb2f4769a638593ffffffffffffffffffffffff0100000001000000 size:0 func:max-bb9c81aa-2cc1-4a8a-983f-194b813792b1 +[GET] object_id:acb2f4769a638593ffffffffffffffffffffffff0100000001000000 size:0 func:min-017f6dc4-2ef3-45fb-8c31-3267cf13321b +[GET] object_id:acb2f4769a638593ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-1-76560f47b9510925df1e3e2bf421692e', 2) +[GET] object_id:d68fec326c8433c9ffffffffffffffffffffffff0100000001000000 size:0 func:max-37bb0297-0823-4764-9ed4-7f075658bfd6 +[GET] object_id:d68fec326c8433c9ffffffffffffffffffffffff0100000001000000 size:0 func:min-d9f40475-208f-4098-aa15-c7b6fc9edbf4 +[GET] object_id:d68fec326c8433c9ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-1-76560f47b9510925df1e3e2bf421692e', 3) +[GET] object_id:909a212b104ea2f1ffffffffffffffffffffffff0100000001000000 size:0 func:max-239fa783-d16b-4bfb-aa6c-81eb999da9c9 +[GET] object_id:909a212b104ea2f1ffffffffffffffffffffffff0100000001000000 size:0 func:min-ab2ad07c-b9de-4e0d-869f-1684563daac4 +[GET] object_id:909a212b104ea2f1ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-1-76560f47b9510925df1e3e2bf421692e', 4) +[GET] object_id:d5a75db31f99bd73ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 0, 1) +[GET] object_id:aa64d94c3d0cf856ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 0, 1) +[GET] object_id:5497aa04f981e4a1ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 0, 1) +[GET] object_id:c96088b12950798bffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 1, 0) +[GET] object_id:4c37da832ee7bc31ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 1, 0) +[GET] object_id:a4b140629d390f70ffffffffffffffffffffffff0100000001000000 size:0 func:max-153ab69d-6e5a-483b-ae4b-ea353f48cac1 +[GET] object_id:a4b140629d390f70ffffffffffffffffffffffff0100000001000000 size:0 func:min-22e57d0c-227f-4b56-9a26-fd61bf800794 +[GET] object_id:a4b140629d390f70ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-1-76560f47b9510925df1e3e2bf421692e', 7) +[GET] object_id:5e4556eab3523b9cffffffffffffffffffffffff0100000001000000 size:0 func:max-54b56d3e-e12f-4a44-a093-0471b01e5823 +[GET] object_id:5e4556eab3523b9cffffffffffffffffffffffff0100000001000000 size:0 func:min-f0cfbaf4-bbab-44ce-9e5b-9652a625f2de +[GET] object_id:5e4556eab3523b9cffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-1-76560f47b9510925df1e3e2bf421692e', 8) +[GET] object_id:8d810c38302b25afffffffffffffffffffffffff0100000001000000 size:0 func:max-032cb695-d1b6-484f-ad8f-54ab388456b4 +[GET] object_id:8d810c38302b25afffffffffffffffffffffffff0100000001000000 size:0 func:min-f5467333-1fa1-4704-b656-cc0c95482466 +[GET] object_id:8d810c38302b25afffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-1-76560f47b9510925df1e3e2bf421692e', 9) +[GET] object_id:21711c35be2858f6ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 0, 3) +[GET] object_id:b4b63d32e46ad812ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 0, 3) +[GET] object_id:e7b04d2c8c844064ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 0, 3) +[GET] object_id:7d174d6864a3d2bbffffffffffffffffffffffff0100000001000000 size:0 func:max-b6bff76c-23ea-4e80-92af-a14ac7c787a1 +[GET] object_id:7d174d6864a3d2bbffffffffffffffffffffffff0100000001000000 size:0 func:min-53ef308e-a157-4728-9908-3dfda742c2fe +[GET] object_id:7d174d6864a3d2bbffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-1-76560f47b9510925df1e3e2bf421692e', 5) +[GET] object_id:ff263408fac73fd7ffffffffffffffffffffffff0100000001000000 size:0 func:max-2c02c2e7-b1e0-4ae6-b355-813d9197c914 +[GET] object_id:ff263408fac73fd7ffffffffffffffffffffffff0100000001000000 size:0 func:min-7c1cc151-85ad-466e-bc65-b26d56c2e9d1 +[GET] object_id:ff263408fac73fd7ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-1-76560f47b9510925df1e3e2bf421692e', 6) +[GET] object_id:19f1295112af6a56ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 0, 2) +[GET] object_id:931690d20f999d2fffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 0, 2) +[GET] object_id:964d85edc3bacb5bffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 1, 1) +[GET] object_id:d3ab1aaf09c00ee8ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 1, 1) +[GET] object_id:3f83646dddac6e52ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 2, 0) +[GET] object_id:b9679322487eb9c3ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 2, 0) +[GET] object_id:9d78bd90898368caffffffffffffffffffffffff0100000001000000 size:0 func:max-ef605214-688c-4895-8a6e-707d863786ac +[GET] object_id:9d78bd90898368caffffffffffffffffffffffff0100000001000000 size:0 func:min-8c5d25ac-0e13-4b85-a349-975dd9da01ac +[GET] object_id:9d78bd90898368caffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-1-76560f47b9510925df1e3e2bf421692e', 14) +[GET] object_id:aae99729138a74d2ffffffffffffffffffffffff0100000001000000 size:0 func:max-29c4da43-7c74-4939-b8dc-3bb592cee9a7 +[GET] object_id:aae99729138a74d2ffffffffffffffffffffffff0100000001000000 size:0 func:min-dc3f5407-887f-4a14-b060-93498eb5ccb9 +[GET] object_id:aae99729138a74d2ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-1-76560f47b9510925df1e3e2bf421692e', 15) +[GET] object_id:2b1d1545893d1daeffffffffffffffffffffffff0100000001000000 size:0 func:max-e4aef3b8-c423-4c78-8810-aa77c808bf74 +[GET] object_id:2b1d1545893d1daeffffffffffffffffffffffff0100000001000000 size:0 func:min-1d6b6f48-49b9-45b9-8229-192a3ba53a67 +[GET] object_id:2b1d1545893d1daeffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-1-76560f47b9510925df1e3e2bf421692e', 16) +[GET] object_id:2638cdc57fda8d43ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 0, 6) +[GET] object_id:9bb55bc9a880f4ceffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 0, 6) +[GET] object_id:faf123d8b1b35293ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 0, 6) +[GET] object_id:e07eebba574c5c52ffffffffffffffffffffffff0100000001000000 size:0 func:max-c9a8c9ce-3785-4fc5-8236-230facb08b1b +[GET] object_id:e07eebba574c5c52ffffffffffffffffffffffff0100000001000000 size:0 func:min-c1ebc0ca-caed-4aab-8611-99c6c20b2f95 +[GET] object_id:e07eebba574c5c52ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-1-76560f47b9510925df1e3e2bf421692e', 17) +[GET] object_id:85186699e206774effffffffffffffffffffffff0100000001000000 size:0 func:max-1f338c52-8004-4662-a5d2-7601d797b061 +[GET] object_id:85186699e206774effffffffffffffffffffffff0100000001000000 size:0 func:min-583f0bf0-2c88-4544-b660-e0fced857a38 +[GET] object_id:85186699e206774effffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-1-76560f47b9510925df1e3e2bf421692e', 18) +[GET] object_id:3f903472d63abe4effffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 0, 7) +[GET] object_id:6c1e0b9bd7e2a3a6ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 0, 7) +[GET] object_id:e707441f8900a635ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 1, 3) +[GET] object_id:02bed209b72c305affffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 1, 3) +[GET] object_id:ed64f45346849778ffffffffffffffffffffffff0100000001000000 size:0 func:max-2bc13a35-ed08-48bc-9f09-c731110d453e +[GET] object_id:ed64f45346849778ffffffffffffffffffffffff0100000001000000 size:0 func:min-8f574a88-df50-4939-aecb-8f9b76738b60 +[GET] object_id:ed64f45346849778ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-1-76560f47b9510925df1e3e2bf421692e', 10) +[GET] object_id:671f1fd57bc3e36effffffffffffffffffffffff0100000001000000 size:0 func:max-498ea2b3-915d-4d10-bef2-56b92d307808 +[GET] object_id:671f1fd57bc3e36effffffffffffffffffffffff0100000001000000 size:0 func:min-e675a038-2871-40ed-b906-2168d4e76aed +[GET] object_id:671f1fd57bc3e36effffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-1-76560f47b9510925df1e3e2bf421692e', 11) +[GET] object_id:5b78680d95855782ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 0, 4) +[GET] object_id:dc81b612c7b1ec30ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 0, 4) +[GET] object_id:6dbd075c28ae20d7ffffffffffffffffffffffff0100000001000000 size:0 func:max-c7645d11-5e0b-40b3-a23c-2459a53c79fb +[GET] object_id:6dbd075c28ae20d7ffffffffffffffffffffffff0100000001000000 size:0 func:min-bccdae83-da30-4bf6-a6b0-020c9f86a923 +[GET] object_id:6dbd075c28ae20d7ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-1-76560f47b9510925df1e3e2bf421692e', 12) +[GET] object_id:59f3350af995668affffffffffffffffffffffff0100000001000000 size:0 func:max-150fcd1c-0f0f-499d-b6a3-ea63d1a61d96 +[GET] object_id:59f3350af995668affffffffffffffffffffffff0100000001000000 size:0 func:min-198419be-7442-4bbd-a528-3c65f8217c0c +[GET] object_id:59f3350af995668affffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-1-76560f47b9510925df1e3e2bf421692e', 13) +[GET] object_id:8dca14b7f0ef35a5ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 0, 5) +[GET] object_id:da16cde2289a1051ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 0, 5) +[GET] object_id:2f41f345ddf68417ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 1, 2) +[GET] object_id:ec7da20ba0a94845ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 1, 2) +[GET] object_id:a7f56d220ef857faffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 2, 1) +[GET] object_id:3a4bb9b95c2938a4ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 2, 1) +[GET] object_id:9e8c0eaa9bab673cffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 3, 0) +[GET] object_id:fc8204747604c8ccffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-2-76560f47b9510925df1e3e2bf421692e', 3, 0) +[GET] object_id:30c04f84db70b40cffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-3-76560f47b9510925df1e3e2bf421692e', 0) +[GET] object_id:bbde8638d39a1245ffffffffffffffffffffffff0100000001000000 size:0 func:('re-quantiles-3-76560f47b9510925df1e3e2bf421692e', 0) +(pid=20860) +(pid=20860) +(pid=20860) +(pid=20863) +(pid=20869) +(pid=20861) +(pid=20871) +(pid=20854) +(pid=20866) +(pid=20863) +(pid=20871) +(pid=20864) +(pid=20874) +(pid=20869) +(pid=20873) +(pid=20867) +(pid=20872) +(pid=20861) +(pid=20854) +(pid=20866) +(pid=20862) +(pid=20864) +(pid=20874) +(pid=20873) +(pid=20867) +(pid=20872) +(pid=20871) +(pid=20874) +(pid=20872) +(pid=20862) +(pid=20870) +(pid=20866) +(pid=20873) +(pid=20867) +(pid=20862) +(pid=20870) +(pid=20867) +(pid=20870) +(pid=20866) +(pid=20873) +(pid=20867) +(pid=20866) +(pid=20863) +(pid=20863) +(pid=20869) +(pid=20863) +(pid=20869) +(pid=20863) +(pid=20854) +(pid=20860) +(pid=20874) +(pid=20872) +(pid=20868) +(pid=20868) +(pid=20868) +(pid=20862) +(pid=20937) +(pid=20959) +(pid=20961) +[GET] object_id:adfbf615e1b4bed2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 0, 0) +[GET] object_id:adfbf615e1b4bed2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 0, 1) +[GET] object_id:adfbf615e1b4bed2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 0, 2) +[GET] object_id:adfbf615e1b4bed2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 0, 3) +[GET] object_id:adfbf615e1b4bed2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 0, 4) +[GET] object_id:adfbf615e1b4bed2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 0, 5) +[GET] object_id:adfbf615e1b4bed2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 0, 6) +[GET] object_id:adfbf615e1b4bed2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 0, 7) +[GET] object_id:adfbf615e1b4bed2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 0, 8) +[GET] object_id:adfbf615e1b4bed2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 0, 9) +[GET] object_id:adfbf615e1b4bed2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 0, 10) +[GET] object_id:adfbf615e1b4bed2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 0, 11) +[GET] object_id:adfbf615e1b4bed2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 0, 12) +[GET] object_id:adfbf615e1b4bed2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 0, 13) +[GET] object_id:adfbf615e1b4bed2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 0, 14) +[GET] object_id:adfbf615e1b4bed2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 0, 15) +[GET] object_id:adfbf615e1b4bed2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 0, 16) +[GET] object_id:adfbf615e1b4bed2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 0, 17) +[GET] object_id:adfbf615e1b4bed2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 0, 18) +[GET] object_id:1512d441ba7a26bcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 1, 0) +[GET] object_id:1512d441ba7a26bcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 1, 1) +[GET] object_id:1512d441ba7a26bcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 1, 2) +[GET] object_id:1512d441ba7a26bcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 1, 3) +[GET] object_id:1512d441ba7a26bcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 1, 4) +[GET] object_id:1512d441ba7a26bcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 1, 5) +[GET] object_id:1512d441ba7a26bcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 1, 6) +[GET] object_id:1512d441ba7a26bcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 1, 7) +[GET] object_id:1512d441ba7a26bcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 1, 8) +[GET] object_id:1512d441ba7a26bcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 1, 9) +[GET] object_id:1512d441ba7a26bcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 1, 10) +[GET] object_id:1512d441ba7a26bcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 1, 11) +[GET] object_id:1512d441ba7a26bcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 1, 12) +[GET] object_id:1512d441ba7a26bcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 1, 13) +[GET] object_id:1512d441ba7a26bcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 1, 14) +[GET] object_id:1512d441ba7a26bcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 1, 15) +[GET] object_id:1512d441ba7a26bcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 1, 16) +[GET] object_id:1512d441ba7a26bcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 1, 17) +[GET] object_id:1512d441ba7a26bcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 1, 18) +[GET] object_id:3a78ec95833a4631ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 2, 0) +[GET] object_id:3a78ec95833a4631ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 2, 1) +[GET] object_id:3a78ec95833a4631ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 2, 2) +[GET] object_id:3a78ec95833a4631ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 2, 3) +[GET] object_id:3a78ec95833a4631ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 2, 4) +[GET] object_id:3a78ec95833a4631ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 2, 5) +[GET] object_id:3a78ec95833a4631ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 2, 6) +[GET] object_id:3a78ec95833a4631ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 2, 7) +[GET] object_id:3a78ec95833a4631ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 2, 8) +[GET] object_id:3a78ec95833a4631ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 2, 9) +[GET] object_id:3a78ec95833a4631ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 2, 10) +[GET] object_id:3a78ec95833a4631ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 2, 11) +[GET] object_id:3a78ec95833a4631ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 2, 12) +[GET] object_id:3a78ec95833a4631ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 2, 13) +[GET] object_id:3a78ec95833a4631ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 2, 14) +[GET] object_id:3a78ec95833a4631ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 2, 15) +[GET] object_id:3a78ec95833a4631ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 2, 16) +[GET] object_id:3a78ec95833a4631ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 2, 17) +[GET] object_id:3a78ec95833a4631ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 2, 18) +[GET] object_id:fddd08b1d1ce0c2bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 3, 0) +[GET] object_id:fddd08b1d1ce0c2bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 3, 1) +[GET] object_id:fddd08b1d1ce0c2bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 3, 2) +[GET] object_id:fddd08b1d1ce0c2bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 3, 3) +[GET] object_id:fddd08b1d1ce0c2bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 3, 4) +[GET] object_id:fddd08b1d1ce0c2bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 3, 5) +[GET] object_id:fddd08b1d1ce0c2bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 3, 6) +[GET] object_id:fddd08b1d1ce0c2bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 3, 7) +[GET] object_id:fddd08b1d1ce0c2bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 3, 8) +[GET] object_id:fddd08b1d1ce0c2bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 3, 9) +[GET] object_id:fddd08b1d1ce0c2bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 3, 10) +[GET] object_id:fddd08b1d1ce0c2bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 3, 11) +[GET] object_id:fddd08b1d1ce0c2bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 3, 12) +[GET] object_id:fddd08b1d1ce0c2bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 3, 13) +[GET] object_id:fddd08b1d1ce0c2bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 3, 14) +[GET] object_id:fddd08b1d1ce0c2bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 3, 15) +[GET] object_id:fddd08b1d1ce0c2bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 3, 16) +[GET] object_id:fddd08b1d1ce0c2bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 3, 17) +[GET] object_id:fddd08b1d1ce0c2bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 3, 18) +[GET] object_id:77cd0cc01feb307dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 4, 0) +[GET] object_id:77cd0cc01feb307dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 4, 1) +[GET] object_id:77cd0cc01feb307dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 4, 2) +[GET] object_id:77cd0cc01feb307dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 4, 3) +[GET] object_id:77cd0cc01feb307dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 4, 4) +[GET] object_id:77cd0cc01feb307dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 4, 5) +[GET] object_id:77cd0cc01feb307dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 4, 6) +[GET] object_id:77cd0cc01feb307dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 4, 7) +[GET] object_id:77cd0cc01feb307dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 4, 8) +[GET] object_id:77cd0cc01feb307dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 4, 9) +[GET] object_id:77cd0cc01feb307dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 4, 10) +[GET] object_id:77cd0cc01feb307dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 4, 11) +[GET] object_id:77cd0cc01feb307dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 4, 12) +[GET] object_id:77cd0cc01feb307dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 4, 13) +[GET] object_id:77cd0cc01feb307dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 4, 14) +[GET] object_id:77cd0cc01feb307dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 4, 15) +[GET] object_id:77cd0cc01feb307dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 4, 16) +[GET] object_id:77cd0cc01feb307dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 4, 17) +[GET] object_id:77cd0cc01feb307dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 4, 18) +[GET] object_id:7a1a4c14e4724eb9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 5, 0) +[GET] object_id:7a1a4c14e4724eb9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 5, 1) +[GET] object_id:7a1a4c14e4724eb9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 5, 2) +[GET] object_id:7a1a4c14e4724eb9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 5, 3) +[GET] object_id:7a1a4c14e4724eb9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 5, 4) +[GET] object_id:7a1a4c14e4724eb9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 5, 5) +[GET] object_id:7a1a4c14e4724eb9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 5, 6) +[GET] object_id:7a1a4c14e4724eb9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 5, 7) +[GET] object_id:7a1a4c14e4724eb9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 5, 8) +[GET] object_id:7a1a4c14e4724eb9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 5, 9) +[GET] object_id:7a1a4c14e4724eb9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 5, 10) +[GET] object_id:7a1a4c14e4724eb9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 5, 11) +[GET] object_id:7a1a4c14e4724eb9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 5, 12) +[GET] object_id:7a1a4c14e4724eb9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 5, 13) +[GET] object_id:7a1a4c14e4724eb9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 5, 14) +[GET] object_id:7a1a4c14e4724eb9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 5, 15) +[GET] object_id:7a1a4c14e4724eb9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 5, 16) +[GET] object_id:7a1a4c14e4724eb9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 5, 17) +[GET] object_id:7a1a4c14e4724eb9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 5, 18) +[GET] object_id:80a7d39af51fd243ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 6, 0) +[GET] object_id:80a7d39af51fd243ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 6, 1) +[GET] object_id:80a7d39af51fd243ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 6, 2) +[GET] object_id:80a7d39af51fd243ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 6, 3) +[GET] object_id:80a7d39af51fd243ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 6, 4) +[GET] object_id:80a7d39af51fd243ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 6, 5) +[GET] object_id:80a7d39af51fd243ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 6, 6) +[GET] object_id:80a7d39af51fd243ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 6, 7) +[GET] object_id:80a7d39af51fd243ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 6, 8) +[GET] object_id:80a7d39af51fd243ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 6, 9) +[GET] object_id:80a7d39af51fd243ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 6, 10) +[GET] object_id:80a7d39af51fd243ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 6, 11) +[GET] object_id:80a7d39af51fd243ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 6, 12) +[GET] object_id:80a7d39af51fd243ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 6, 13) +[GET] object_id:80a7d39af51fd243ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 6, 14) +[GET] object_id:80a7d39af51fd243ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 6, 15) +[GET] object_id:80a7d39af51fd243ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 6, 16) +[GET] object_id:80a7d39af51fd243ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 6, 17) +[GET] object_id:80a7d39af51fd243ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 6, 18) +[GET] object_id:cbc4c75aba9becacffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 7, 0) +[GET] object_id:cbc4c75aba9becacffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 7, 1) +[GET] object_id:cbc4c75aba9becacffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 7, 2) +[GET] object_id:cbc4c75aba9becacffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 7, 3) +[GET] object_id:cbc4c75aba9becacffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 7, 4) +[GET] object_id:cbc4c75aba9becacffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 7, 5) +[GET] object_id:cbc4c75aba9becacffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 7, 6) +[GET] object_id:cbc4c75aba9becacffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 7, 7) +[GET] object_id:cbc4c75aba9becacffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 7, 8) +[GET] object_id:cbc4c75aba9becacffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 7, 9) +[GET] object_id:cbc4c75aba9becacffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 7, 10) +[GET] object_id:cbc4c75aba9becacffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 7, 11) +[GET] object_id:cbc4c75aba9becacffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 7, 12) +[GET] object_id:cbc4c75aba9becacffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 7, 13) +[GET] object_id:cbc4c75aba9becacffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 7, 14) +[GET] object_id:cbc4c75aba9becacffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 7, 15) +[GET] object_id:cbc4c75aba9becacffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 7, 16) +[GET] object_id:cbc4c75aba9becacffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 7, 17) +[GET] object_id:cbc4c75aba9becacffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 7, 18) +[GET] object_id:d0fd7f95ef697c7cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 8, 0) +[GET] object_id:d0fd7f95ef697c7cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 8, 1) +[GET] object_id:d0fd7f95ef697c7cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 8, 2) +[GET] object_id:d0fd7f95ef697c7cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 8, 3) +[GET] object_id:d0fd7f95ef697c7cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 8, 4) +[GET] object_id:d0fd7f95ef697c7cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 8, 5) +[GET] object_id:d0fd7f95ef697c7cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 8, 6) +[GET] object_id:d0fd7f95ef697c7cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 8, 7) +[GET] object_id:d0fd7f95ef697c7cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 8, 8) +[GET] object_id:d0fd7f95ef697c7cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 8, 9) +[GET] object_id:d0fd7f95ef697c7cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 8, 10) +[GET] object_id:d0fd7f95ef697c7cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 8, 11) +[GET] object_id:d0fd7f95ef697c7cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 8, 12) +[GET] object_id:d0fd7f95ef697c7cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 8, 13) +[GET] object_id:d0fd7f95ef697c7cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 8, 14) +[GET] object_id:d0fd7f95ef697c7cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 8, 15) +[GET] object_id:d0fd7f95ef697c7cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 8, 16) +[GET] object_id:d0fd7f95ef697c7cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 8, 17) +[GET] object_id:d0fd7f95ef697c7cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 8, 18) +[GET] object_id:cdb8650bb9623158ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 9, 0) +[GET] object_id:cdb8650bb9623158ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 9, 1) +[GET] object_id:cdb8650bb9623158ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 9, 2) +[GET] object_id:cdb8650bb9623158ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 9, 3) +[GET] object_id:cdb8650bb9623158ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 9, 4) +[GET] object_id:cdb8650bb9623158ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 9, 5) +[GET] object_id:cdb8650bb9623158ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 9, 6) +[GET] object_id:cdb8650bb9623158ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 9, 7) +[GET] object_id:cdb8650bb9623158ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 9, 8) +[GET] object_id:cdb8650bb9623158ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 9, 9) +[GET] object_id:cdb8650bb9623158ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 9, 10) +[GET] object_id:cdb8650bb9623158ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 9, 11) +[GET] object_id:cdb8650bb9623158ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 9, 12) +[GET] object_id:cdb8650bb9623158ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 9, 13) +[GET] object_id:cdb8650bb9623158ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 9, 14) +[GET] object_id:cdb8650bb9623158ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 9, 15) +[GET] object_id:cdb8650bb9623158ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 9, 16) +[GET] object_id:cdb8650bb9623158ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 9, 17) +[GET] object_id:cdb8650bb9623158ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 9, 18) +[GET] object_id:679b854130b8d16fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 10, 0) +[GET] object_id:679b854130b8d16fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 10, 1) +[GET] object_id:679b854130b8d16fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 10, 2) +[GET] object_id:679b854130b8d16fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 10, 3) +[GET] object_id:679b854130b8d16fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 10, 4) +[GET] object_id:679b854130b8d16fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 10, 5) +[GET] object_id:679b854130b8d16fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 10, 6) +[GET] object_id:679b854130b8d16fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 10, 7) +[GET] object_id:679b854130b8d16fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 10, 8) +[GET] object_id:679b854130b8d16fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 10, 9) +[GET] object_id:679b854130b8d16fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 10, 10) +[GET] object_id:679b854130b8d16fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 10, 11) +[GET] object_id:679b854130b8d16fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 10, 12) +[GET] object_id:679b854130b8d16fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 10, 13) +[GET] object_id:679b854130b8d16fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 10, 14) +[GET] object_id:679b854130b8d16fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 10, 15) +[GET] object_id:679b854130b8d16fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 10, 16) +[GET] object_id:679b854130b8d16fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 10, 17) +[GET] object_id:679b854130b8d16fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 10, 18) +[GET] object_id:e2cc253491b3402cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 11, 0) +[GET] object_id:e2cc253491b3402cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 11, 1) +[GET] object_id:e2cc253491b3402cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 11, 2) +[GET] object_id:e2cc253491b3402cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 11, 3) +[GET] object_id:e2cc253491b3402cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 11, 4) +[GET] object_id:e2cc253491b3402cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 11, 5) +[GET] object_id:e2cc253491b3402cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 11, 6) +[GET] object_id:e2cc253491b3402cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 11, 7) +[GET] object_id:e2cc253491b3402cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 11, 8) +[GET] object_id:e2cc253491b3402cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 11, 9) +[GET] object_id:e2cc253491b3402cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 11, 10) +[GET] object_id:e2cc253491b3402cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 11, 11) +[GET] object_id:e2cc253491b3402cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 11, 12) +[GET] object_id:e2cc253491b3402cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 11, 13) +[GET] object_id:e2cc253491b3402cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 11, 14) +[GET] object_id:e2cc253491b3402cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 11, 15) +[GET] object_id:e2cc253491b3402cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 11, 16) +[GET] object_id:e2cc253491b3402cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 11, 17) +[GET] object_id:e2cc253491b3402cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 11, 18) +[GET] object_id:ba0de6a92604a456ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 12, 0) +[GET] object_id:ba0de6a92604a456ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 12, 1) +[GET] object_id:ba0de6a92604a456ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 12, 2) +[GET] object_id:ba0de6a92604a456ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 12, 3) +[GET] object_id:ba0de6a92604a456ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 12, 4) +[GET] object_id:ba0de6a92604a456ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 12, 5) +[GET] object_id:ba0de6a92604a456ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 12, 6) +[GET] object_id:ba0de6a92604a456ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 12, 7) +[GET] object_id:ba0de6a92604a456ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 12, 8) +[GET] object_id:ba0de6a92604a456ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 12, 9) +[GET] object_id:ba0de6a92604a456ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 12, 10) +[GET] object_id:ba0de6a92604a456ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 12, 11) +[GET] object_id:ba0de6a92604a456ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 12, 12) +[GET] object_id:ba0de6a92604a456ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 12, 13) +[GET] object_id:ba0de6a92604a456ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 12, 14) +[GET] object_id:ba0de6a92604a456ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 12, 15) +[GET] object_id:ba0de6a92604a456ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 12, 16) +[GET] object_id:ba0de6a92604a456ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 12, 17) +[GET] object_id:ba0de6a92604a456ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 12, 18) +[GET] object_id:aee6d88fc2f569f9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 13, 0) +[GET] object_id:aee6d88fc2f569f9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 13, 1) +[GET] object_id:aee6d88fc2f569f9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 13, 2) +[GET] object_id:aee6d88fc2f569f9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 13, 3) +[GET] object_id:aee6d88fc2f569f9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 13, 4) +[GET] object_id:aee6d88fc2f569f9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 13, 5) +[GET] object_id:aee6d88fc2f569f9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 13, 6) +[GET] object_id:aee6d88fc2f569f9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 13, 7) +[GET] object_id:aee6d88fc2f569f9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 13, 8) +[GET] object_id:aee6d88fc2f569f9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 13, 9) +[GET] object_id:aee6d88fc2f569f9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 13, 10) +[GET] object_id:aee6d88fc2f569f9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 13, 11) +[GET] object_id:aee6d88fc2f569f9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 13, 12) +[GET] object_id:aee6d88fc2f569f9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 13, 13) +[GET] object_id:aee6d88fc2f569f9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 13, 14) +[GET] object_id:aee6d88fc2f569f9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 13, 15) +[GET] object_id:aee6d88fc2f569f9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 13, 16) +[GET] object_id:aee6d88fc2f569f9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 13, 17) +[GET] object_id:aee6d88fc2f569f9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 13, 18) +[GET] object_id:671ea7d5322daa9affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 14, 0) +[GET] object_id:671ea7d5322daa9affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 14, 1) +[GET] object_id:671ea7d5322daa9affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 14, 2) +[GET] object_id:671ea7d5322daa9affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 14, 3) +[GET] object_id:671ea7d5322daa9affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 14, 4) +[GET] object_id:671ea7d5322daa9affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 14, 5) +[GET] object_id:671ea7d5322daa9affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 14, 6) +[GET] object_id:671ea7d5322daa9affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 14, 7) +[GET] object_id:671ea7d5322daa9affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 14, 8) +[GET] object_id:671ea7d5322daa9affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 14, 9) +[GET] object_id:671ea7d5322daa9affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 14, 10) +[GET] object_id:671ea7d5322daa9affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 14, 11) +[GET] object_id:671ea7d5322daa9affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 14, 12) +[GET] object_id:671ea7d5322daa9affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 14, 13) +[GET] object_id:671ea7d5322daa9affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 14, 14) +[GET] object_id:671ea7d5322daa9affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 14, 15) +[GET] object_id:671ea7d5322daa9affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 14, 16) +[GET] object_id:671ea7d5322daa9affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 14, 17) +[GET] object_id:671ea7d5322daa9affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 14, 18) +[GET] object_id:a52ba33d9a5f8757ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 15, 0) +[GET] object_id:a52ba33d9a5f8757ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 15, 1) +[GET] object_id:a52ba33d9a5f8757ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 15, 2) +[GET] object_id:a52ba33d9a5f8757ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 15, 3) +[GET] object_id:a52ba33d9a5f8757ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 15, 4) +[GET] object_id:a52ba33d9a5f8757ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 15, 5) +[GET] object_id:a52ba33d9a5f8757ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 15, 6) +[GET] object_id:a52ba33d9a5f8757ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 15, 7) +[GET] object_id:a52ba33d9a5f8757ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 15, 8) +[GET] object_id:a52ba33d9a5f8757ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 15, 9) +[GET] object_id:a52ba33d9a5f8757ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 15, 10) +[GET] object_id:a52ba33d9a5f8757ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 15, 11) +[GET] object_id:a52ba33d9a5f8757ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 15, 12) +[GET] object_id:a52ba33d9a5f8757ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 15, 13) +[GET] object_id:a52ba33d9a5f8757ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 15, 14) +[GET] object_id:a52ba33d9a5f8757ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 15, 15) +[GET] object_id:a52ba33d9a5f8757ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 15, 16) +[GET] object_id:a52ba33d9a5f8757ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 15, 17) +[GET] object_id:a52ba33d9a5f8757ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 15, 18) +[GET] object_id:8dae4e8559497acfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 16, 0) +[GET] object_id:8dae4e8559497acfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 16, 1) +[GET] object_id:8dae4e8559497acfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 16, 2) +[GET] object_id:8dae4e8559497acfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 16, 3) +[GET] object_id:8dae4e8559497acfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 16, 4) +[GET] object_id:8dae4e8559497acfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 16, 5) +[GET] object_id:8dae4e8559497acfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 16, 6) +[GET] object_id:8dae4e8559497acfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 16, 7) +[GET] object_id:8dae4e8559497acfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 16, 8) +[GET] object_id:8dae4e8559497acfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 16, 9) +[GET] object_id:8dae4e8559497acfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 16, 10) +[GET] object_id:8dae4e8559497acfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 16, 11) +[GET] object_id:8dae4e8559497acfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 16, 12) +[GET] object_id:8dae4e8559497acfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 16, 13) +[GET] object_id:8dae4e8559497acfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 16, 14) +[GET] object_id:8dae4e8559497acfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 16, 15) +[GET] object_id:8dae4e8559497acfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 16, 16) +[GET] object_id:8dae4e8559497acfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 16, 17) +[GET] object_id:8dae4e8559497acfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 16, 18) +[GET] object_id:e69ed3762d1a7fe1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 17, 0) +[GET] object_id:e69ed3762d1a7fe1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 17, 1) +[GET] object_id:e69ed3762d1a7fe1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 17, 2) +[GET] object_id:e69ed3762d1a7fe1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 17, 3) +[GET] object_id:e69ed3762d1a7fe1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 17, 4) +[GET] object_id:e69ed3762d1a7fe1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 17, 5) +[GET] object_id:e69ed3762d1a7fe1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 17, 6) +[GET] object_id:e69ed3762d1a7fe1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 17, 7) +[GET] object_id:e69ed3762d1a7fe1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 17, 8) +[GET] object_id:e69ed3762d1a7fe1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 17, 9) +[GET] object_id:e69ed3762d1a7fe1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 17, 10) +[GET] object_id:e69ed3762d1a7fe1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 17, 11) +[GET] object_id:e69ed3762d1a7fe1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 17, 12) +[GET] object_id:e69ed3762d1a7fe1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 17, 13) +[GET] object_id:e69ed3762d1a7fe1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 17, 14) +[GET] object_id:e69ed3762d1a7fe1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 17, 15) +[GET] object_id:e69ed3762d1a7fe1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 17, 16) +[GET] object_id:e69ed3762d1a7fe1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 17, 17) +[GET] object_id:e69ed3762d1a7fe1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 17, 18) +[GET] object_id:5df04495cec55ef7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 18, 0) +[GET] object_id:6dcffd2ea8538809ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:a52e5caf016c1aebffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:08b4d457f4766763ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:44e5abcd85323f8effffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:e73d369b79cdebbeffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:bcc0a0059e051954ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:76b987029312f36bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:d68840d2161b10fbffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:a150357784540d7bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:ef9b9a36effe4b93ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:620722e228aeb681ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:29424dcd8932a4c8ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:8aefca26bcd28616ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:c361e4b8895ba43cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:9ce1987488ca1394ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:016750f70cca0b68ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:1ef30302d266aa1cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:94913c6c0db0c39dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:f61ad9e5813e66eaffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:5df04495cec55ef7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 18, 1) +[GET] object_id:9ae0efe681cc6e6bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 1) +[GET] object_id:e625fb24a67e7f4affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 1) +[GET] object_id:6805d00f1ed4c4d0ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 1) +[GET] object_id:791bebe9586dc56fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 1) +[GET] object_id:d2bd9bd81e88a8ceffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 1) +[GET] object_id:f65fedf31389fc68ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 1) +[GET] object_id:5ad81f2b86312e76ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 1) +[GET] object_id:888269c1c6147dc9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 1) +[GET] object_id:117795589b3df8deffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 1) +[GET] object_id:2f380fa819400849ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 1) +[GET] object_id:6bbcc232a7988725ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 1) +[GET] object_id:0a733a035bd79564ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 1) +[GET] object_id:166b6437247067d6ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 1) +[GET] object_id:edf5de839b4ef7dcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 1) +[GET] object_id:b27aaece1ffa70d8ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 1) +[GET] object_id:cd02104ee5868014ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 1) +[GET] object_id:934d17eb924805bbffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 1) +[GET] object_id:8f743a1ec7f82ce0ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 1) +[GET] object_id:bd63ac0cf00193deffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 1) +[GET] object_id:5df04495cec55ef7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 18, 2) +[GET] object_id:e53dee1a555c8c2dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 2) +[GET] object_id:2004b0ca36764e4fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 2) +[GET] object_id:b07caca2a282f04effffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 2) +[GET] object_id:92f10cea97f12e5dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 2) +[GET] object_id:9796eb2c92ed28ebffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 2) +[GET] object_id:a50733b41dd6bcdeffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 2) +[GET] object_id:25b29c22c19f79d2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 2) +[GET] object_id:77feb1abdfe96addffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 2) +[GET] object_id:d45dcfabd1ed85e1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 2) +[GET] object_id:d3ec2e91753e4340ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 2) +[GET] object_id:68ecb7b4eb086e6affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 2) +[GET] object_id:a588627715887834ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 2) +[GET] object_id:e5509bd8151f6f42ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 2) +[GET] object_id:a06d247da28159b7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 2) +[GET] object_id:0c94a57c0ef0c230ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 2) +[GET] object_id:47254e8df26338eaffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 2) +[GET] object_id:62a6a9e96de6b60fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 2) +[GET] object_id:227e9b3fbd5d2710ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 2) +[GET] object_id:cd4bcac32f11f9bbffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 2) +[GET] object_id:5df04495cec55ef7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 18, 3) +[GET] object_id:41060ca90473b59fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 3) +[GET] object_id:ccc6942ba2b04ebbffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 3) +[GET] object_id:06c82ce3c4361befffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 3) +[GET] object_id:21ce60b436cef633ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 3) +[GET] object_id:0fb939667ee64409ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 3) +[GET] object_id:21d441dcf87c697fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 3) +[GET] object_id:5900a9f03f81e014ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 3) +[GET] object_id:40aed905f700f037ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 3) +[GET] object_id:e310f8a190d13b09ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 3) +[GET] object_id:ef01aff961e140ffffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 3) +[GET] object_id:5a3df18d8ff85a41ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 3) +[GET] object_id:2948f769827ca076ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 3) +[GET] object_id:3e63a51020b5b425ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 3) +[GET] object_id:b262000626b699cfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 3) +[GET] object_id:3a28bf2b70f864a0ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 3) +[GET] object_id:7f8daad1557c1075ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 3) +[GET] object_id:b5a580ab80c95749ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 3) +[GET] object_id:ed869e5100f3a7e2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 3) +[GET] object_id:6bb4c20a920130f0ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 3) +[GET] object_id:5df04495cec55ef7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 18, 4) +[GET] object_id:3ffc45893d3b0f00ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 4) +[GET] object_id:2fe79181c2b3eb83ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 4) +[GET] object_id:9f9449a8cc6f305dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 4) +[GET] object_id:157d304db8037db2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 4) +[GET] object_id:7c4496026a641650ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 4) +[GET] object_id:c41412f8290c4b4affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 4) +[GET] object_id:ed4ec686f4ac6721ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 4) +[GET] object_id:6de16a1a317ffb76ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 4) +[GET] object_id:de7096f52ff8a6a6ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 4) +[GET] object_id:1e39a44831964be3ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 4) +[GET] object_id:a14ba7d472277b18ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 4) +[GET] object_id:22bc61b5ae4d3a8effffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 4) +[GET] object_id:27770fc93ff61bc2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 4) +[GET] object_id:1cb2082d415f8052ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 4) +[GET] object_id:52776386fecbac7affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 4) +[GET] object_id:de45d7128577d2d2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 4) +[GET] object_id:c99cfbd5e04cb257ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 4) +[GET] object_id:1ef5eb26e201102dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 4) +[GET] object_id:54df1383042b3236ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 4) +[GET] object_id:5df04495cec55ef7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 18, 5) +[GET] object_id:4b633dd797688ed8ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 5) +[GET] object_id:680bcac5eaead5beffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 5) +[GET] object_id:efbcb8a6da6342d3ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 5) +[GET] object_id:ab2496e09b1a2120ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 5) +[GET] object_id:642ef4cf0728b1a1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 5) +[GET] object_id:9b42588d39a120faffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 5) +[GET] object_id:4166bfa78d3afb3bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 5) +[GET] object_id:1675e438f1395ac9ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 5) +[GET] object_id:efc712d65a7d7925ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 5) +[GET] object_id:4817e60e48bb672affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 5) +[GET] object_id:d2c2136d584032f6ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 5) +[GET] object_id:e2b871b0dcae5023ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 5) +[GET] object_id:36a95edd4fac8d4cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 5) +[GET] object_id:dd0d466a92e9d055ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 5) +[GET] object_id:b19d0cb50d2bae9dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 5) +[GET] object_id:7ec544d28d1401daffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 5) +[GET] object_id:43bdcf5f0df9124cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 5) +[GET] object_id:69a44463d6ec2f20ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 5) +[GET] object_id:37b5a77f9f059b21ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 5) +[GET] object_id:5df04495cec55ef7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 18, 6) +[GET] object_id:31ffddd68e69d833ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 6) +[GET] object_id:4b7db0ebf528096dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 6) +[GET] object_id:d406c78e12b22be3ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 6) +[GET] object_id:c2bc738df8622466ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 6) +[GET] object_id:0c8bd26d89afdcffffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 6) +[GET] object_id:d4cd095eff01a11fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 6) +[GET] object_id:a694157133074c15ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 6) +[GET] object_id:f2fc8469c1e2df51ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 6) +[GET] object_id:6e12118fa4238458ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 6) +[GET] object_id:fa1dfe163b57a589ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 6) +[GET] object_id:548d462135bd62f2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 6) +[GET] object_id:29f1bc73b169be7fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 6) +[GET] object_id:72bfe99d18ae8736ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 6) +[GET] object_id:0e5eb7a251c684d3ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 6) +[GET] object_id:83670e116136ff3dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 6) +[GET] object_id:13a6a2c1c586d589ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 6) +[GET] object_id:673945b704209c49ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 6) +[GET] object_id:aac4ca32fcd13c59ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 6) +[GET] object_id:5c7d3b08a4cdf40bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 6) +[GET] object_id:5df04495cec55ef7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 18, 7) +[GET] object_id:02f9c3a971058d26ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 7) +[GET] object_id:73158d7e0630b425ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 7) +[GET] object_id:15f6a6a8b66dbb73ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 7) +[GET] object_id:7dc7d3813fd41ea0ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 7) +[GET] object_id:d9550f926994eccdffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 7) +[GET] object_id:fdc50cb1e611fce6ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 7) +[GET] object_id:c9b56097ccb17980ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 7) +[GET] object_id:a1cfcd13afac39deffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 7) +[GET] object_id:46766c5b0167bafaffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 7) +[GET] object_id:096590c4496b9db1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 7) +[GET] object_id:96622d82e0eb1eb6ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 7) +[GET] object_id:a4f4445349906f5dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 7) +[GET] object_id:6518258f41da3feeffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 7) +[GET] object_id:d04d5aca628a2e8dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 7) +[GET] object_id:d26830043e2fe7cfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 7) +[GET] object_id:da417cf8b9a19d09ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 7) +[GET] object_id:3cbb460a53b9781affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 7) +[GET] object_id:1a99868d379af149ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 7) +[GET] object_id:247ca55dc2ee56e4ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 7) +[GET] object_id:5df04495cec55ef7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 18, 8) +[GET] object_id:55ecd27aff6f4d98ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 8) +[GET] object_id:9b3f9d97a518836fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 8) +[GET] object_id:92d4c064f994815effffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 8) +[GET] object_id:57d8d080d7906c5effffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 8) +[GET] object_id:90fd018037cd8211ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 8) +[GET] object_id:8d702469aacc79ceffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 8) +[GET] object_id:48aa69d2baea66fcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 8) +[GET] object_id:367778066b8cd3c3ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 8) +[GET] object_id:beeb98b72e2c2f40ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 8) +[GET] object_id:e16d9fa70dd43343ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 8) +[GET] object_id:99e62cd984b8c7deffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 8) +[GET] object_id:f5e82cb3be06ae85ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 8) +[GET] object_id:db8269cc8098b69effffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 8) +[GET] object_id:7916f59a41ca43e7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 8) +[GET] object_id:55d638535f2eec69ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 8) +[GET] object_id:5d51c9acaefe7611ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 8) +[GET] object_id:147d4911e0333545ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 8) +[GET] object_id:8f4558a0dc0fcff3ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 8) +[GET] object_id:1aa22206bc0abbb2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 8) +[GET] object_id:5df04495cec55ef7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 18, 9) +[GET] object_id:e756911eccfb221bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 9) +[GET] object_id:089598eaf7c4989cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 9) +[GET] object_id:70258a3ec1580273ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 9) +[GET] object_id:f7dc3a80d7a578ceffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 9) +[GET] object_id:9d22d7133452b1ffffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 9) +[GET] object_id:a6c6f26acd330281ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 9) +[GET] object_id:c4d13c5ad1cffac0ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 9) +[GET] object_id:c8c4834ecd27e853ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 9) +[GET] object_id:b312bd42f7d7400bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 9) +[GET] object_id:9637a08885cb3125ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 9) +[GET] object_id:059f1b6e0af7350bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 9) +[GET] object_id:d6be6c0e2ea6df93ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 9) +[GET] object_id:d6bba9a16d016a41ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 9) +[GET] object_id:a95c649438f790a8ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 9) +[GET] object_id:21ea3787251444f1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 9) +[GET] object_id:c3a51edce6322cb0ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 9) +[GET] object_id:fdd68afa3b96489fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 9) +[GET] object_id:e3caa05291e63656ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 9) +[GET] object_id:1b81126874d80e22ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 9) +[GET] object_id:5df04495cec55ef7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 18, 10) +[GET] object_id:0ce0d47662826002ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 10) +[GET] object_id:7346fcdbeb9b47a8ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 10) +[GET] object_id:735730a1bdf733b7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 10) +[GET] object_id:72aff9355831c99cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 10) +[GET] object_id:e5eb40d302a6fdfaffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 10) +[GET] object_id:17394a99af3d4870ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 10) +[GET] object_id:a4030b880e139ab1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 10) +[GET] object_id:4a3eb84642a4eba1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 10) +[GET] object_id:01249b40d3e146efffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 10) +[GET] object_id:b646c5fab074f6a7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 10) +[GET] object_id:cae5a6086de2c8e8ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 10) +[GET] object_id:9a4aa5b17820f0f2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 10) +[GET] object_id:580669f9d13bb794ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 10) +[GET] object_id:9f8d7f46a2c850cfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 10) +[GET] object_id:6ae62c2131af58aeffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 10) +[GET] object_id:1e196b9bf3f4a2b4ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 10) +[GET] object_id:aac918c9309d6aaaffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 10) +[GET] object_id:ce4942442b37c656ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 10) +[GET] object_id:b361d7a36076b6a6ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 10) +[GET] object_id:5df04495cec55ef7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 18, 11) +[GET] object_id:fae0588f3c63d706ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 11) +[GET] object_id:ec49c09ab8d5ba02ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 11) +[GET] object_id:39abb479acfc261bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 11) +[GET] object_id:02f9b42dcf9096dbffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 11) +[GET] object_id:1a9b358a96a3682bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 11) +[GET] object_id:2855a357e1acad6fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 11) +[GET] object_id:2576f621731882ccffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 11) +[GET] object_id:3c87d10b55f3a5dbffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 11) +[GET] object_id:76e64f4e82017f3dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 11) +[GET] object_id:dd8c981f6f8cfeecffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 11) +[GET] object_id:ad833e1dde152e45ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 11) +[GET] object_id:ace8226f502d2727ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 11) +[GET] object_id:2a06d5f159ab1b64ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 11) +[GET] object_id:5dc7964f535d2ec0ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 11) +[GET] object_id:59f76feddb5a7dbcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 11) +[GET] object_id:298d6588cc3f37a5ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 11) +[GET] object_id:c999206b101feea5ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 11) +[GET] object_id:7d118de670c46e28ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 11) +[GET] object_id:a18efc956cc795cbffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 11) +[GET] object_id:5df04495cec55ef7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 18, 12) +[GET] object_id:53a9d7821ab2b23cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 12) +[GET] object_id:b7d13c88df3796a5ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 12) +[GET] object_id:a6fa3713c31d6018ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 12) +[GET] object_id:8b6527a94d581e29ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 12) +[GET] object_id:9ddb92cdccb89689ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 12) +[GET] object_id:22c467953099947fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 12) +[GET] object_id:b01ef3d33cc89dcdffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 12) +[GET] object_id:1d5f2724e9372d75ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 12) +[GET] object_id:2b7f9b8463d77440ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 12) +[GET] object_id:db3ae28ea3d091dbffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 12) +[GET] object_id:c6d63e63f383933affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 12) +[GET] object_id:885b68732fc24eacffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 12) +[GET] object_id:cd17d17a5e18c3eaffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 12) +[GET] object_id:dc3689fed918fc13ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 12) +[GET] object_id:7dbdaf4807e09f2fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 12) +[GET] object_id:d884268a775f1c87ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 12) +[GET] object_id:b8f56bba0f717822ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 12) +[GET] object_id:950129ab022be21cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 12) +[GET] object_id:24a16b24332d04c0ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 12) +[GET] object_id:5df04495cec55ef7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 18, 13) +[GET] object_id:d74ee889103a0be6ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 13) +[GET] object_id:9d4a0ce33c206278ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 13) +[GET] object_id:df0eabc280c138e3ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 13) +[GET] object_id:1be8bee2ab271f9effffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 13) +[GET] object_id:265894504cc45081ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 13) +[GET] object_id:29c8f0f7467150d5ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 13) +[GET] object_id:fc51829ebaba45e2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 13) +[GET] object_id:b904200e39b72e25ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 13) +[GET] object_id:e7f5754ee0520eb8ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 13) +[GET] object_id:7ae0d96ae1fe63d5ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 13) +[GET] object_id:a5ae88a63853199dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 13) +[GET] object_id:111b9d7c06cc07d6ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 13) +[GET] object_id:a85ba13458d26c05ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 13) +[GET] object_id:e88c868b07d5ab9affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 13) +[GET] object_id:9e54bf54e931a514ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 13) +[GET] object_id:eb630dc5fe1749b6ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 13) +[GET] object_id:1cd33d774c98ae5cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 13) +[GET] object_id:0d58ca55ef31ed03ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 13) +[GET] object_id:419d38867099fbd8ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 13) +[GET] object_id:5df04495cec55ef7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 18, 14) +[GET] object_id:8d0e13b54e656501ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 14) +[GET] object_id:fcb9108fa1a50eefffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 14) +[GET] object_id:c0a406db0e8868b2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 14) +[GET] object_id:97f1f045efc433a6ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 14) +[GET] object_id:2969e0b2d397af6cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 14) +[GET] object_id:2ba2de5a059b7c7affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 14) +[GET] object_id:da4ea3d0142b0716ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 14) +[GET] object_id:ad2654ac589a3f88ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 14) +[GET] object_id:099618e5e640613effffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 14) +[GET] object_id:614f151cd0cce98fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 14) +[GET] object_id:b95c70a98975ff01ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 14) +[GET] object_id:5e47276ce953c241ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 14) +[GET] object_id:5b222b7c04dcac35ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 14) +[GET] object_id:e691d7a91d3becdcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 14) +[GET] object_id:eae47e67369247d5ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 14) +[GET] object_id:5fb55ef7a1c498f7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 14) +[GET] object_id:1e4d649dabc0933fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 14) +[GET] object_id:5c2fb81b921bed52ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 14) +[GET] object_id:2d822426c469ff7fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 14) +[GET] object_id:5df04495cec55ef7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 18, 15) +[GET] object_id:e0f89b42d2212012ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 15) +[GET] object_id:d1e6badb23c63eadffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 15) +[GET] object_id:a38a8afce179207fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 15) +[GET] object_id:20ed6ab43771a5abffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 15) +[GET] object_id:a8ed4b4d8cfd21cfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 15) +[GET] object_id:b5189c7e8f749dedffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 15) +[GET] object_id:a15e43fc986a9597ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 15) +[GET] object_id:86a09f545d0aa88effffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 15) +[GET] object_id:7f89465d47c8d5dcffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 15) +[GET] object_id:d695335b811eab9bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 15) +[GET] object_id:94d7a9c5cbb282e4ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 15) +[GET] object_id:632d55426a7b7b87ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 15) +[GET] object_id:1de3d62ecc441e2dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 15) +[GET] object_id:6bdaf0e7a9e61a4cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 15) +[GET] object_id:368ad6338ba18dc2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 15) +[GET] object_id:e7c2a01b063e9b9bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 15) +[GET] object_id:3bc6df63f4a4b6efffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 15) +[GET] object_id:0a18f9286e7f0a3dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 15) +[GET] object_id:a47c7afa02144811ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 15) +[GET] object_id:5df04495cec55ef7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 18, 16) +[GET] object_id:33f42d85f16d12f5ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 16) +[GET] object_id:a8a1349f6670b3eeffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 16) +[GET] object_id:4f9bf309674f4618ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 16) +[GET] object_id:a770bdf6f225725affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 16) +[GET] object_id:485cd1e3cfb4af77ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 16) +[GET] object_id:15e55bc144d0e5b0ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 16) +[GET] object_id:2421976196c82da8ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 16) +[GET] object_id:4eb7fab6e5030545ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 16) +[GET] object_id:8e178e1ce55c1466ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 16) +[GET] object_id:919a45ef3888a87affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 16) +[GET] object_id:b712c9a2442f326effffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 16) +[GET] object_id:20d3f3cc944a6233ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 16) +[GET] object_id:623284dacf2339f1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 16) +[GET] object_id:90980d3f030b4b92ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 16) +[GET] object_id:7acd8f04079de4e5ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 16) +[GET] object_id:c2d4c915a2db0886ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 16) +[GET] object_id:8ac2990a3c89d814ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 16) +[GET] object_id:f72915c3acda6455ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 16) +[GET] object_id:578e6f8acc7a007cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 16) +[GET] object_id:5df04495cec55ef7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 18, 17) +[GET] object_id:1527e7240dfab35cffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 17) +[GET] object_id:27b757da15f99fd8ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 17) +[GET] object_id:7deea89df8160501ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 17) +[GET] object_id:c2b29e800532c39affffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 17) +[GET] object_id:68e02d92be6ddfccffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 17) +[GET] object_id:ce2397de373e5655ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 17) +[GET] object_id:92ecfb2aa0841af2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 17) +[GET] object_id:af7f51af57eafe69ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 17) +[GET] object_id:d30cb9a3c39d57c6ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 17) +[GET] object_id:ba82e993d311d8cfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 17) +[GET] object_id:47d445c93c6a7903ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 17) +[GET] object_id:1a8fda88e1e8f001ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 17) +[GET] object_id:27232d9cef8063cdffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 17) +[GET] object_id:b54503e2705f85ddffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 17) +[GET] object_id:af444172a3cb9e20ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 17) +[GET] object_id:2c2f630d68d8a7c2ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 17) +[GET] object_id:cdcaca9b2d0112c5ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 17) +[GET] object_id:dbda44882cfe1588ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 17) +[GET] object_id:9e2738050a3c62a6ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 17) +[GET] object_id:5df04495cec55ef7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-split-ca93e1101d7c38c2d1778c94e785b328', 18, 18) +[GET] object_id:93b3620db06dc1ddffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 18) +[GET] object_id:73f5103c34b0b0f1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 18) +[GET] object_id:828348fce69e3057ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 18) +[GET] object_id:a897191171b7ace1ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 18) +[GET] object_id:34b3aa64ed536d8fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 18) +[GET] object_id:11392c3a7557aa2dffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 18) +[GET] object_id:4ddf996cc45c7de5ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 18) +[GET] object_id:f51eec5635a84bcfffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 18) +[GET] object_id:fecb8e8d51a30f07ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 18) +[GET] object_id:b0a2ad0e40698d00ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 18) +[GET] object_id:04e5f469f154edf7ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 18) +[GET] object_id:c96542f1ac0a36e8ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 18) +[GET] object_id:4b206361a8c224ecffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 18) +[GET] object_id:f74e3e0219621f0bffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 18) +[GET] object_id:ea50a9a22a3b0d9fffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 18) +[GET] object_id:f05254974f3565c3ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 18) +[GET] object_id:6dc8af0b4b33bcfaffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 18) +[GET] object_id:9a15bc1edd638498ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 18) +[GET] object_id:9b340ff7790f4bb0ffffffffffffffffffffffff0100000001000000 size:0 func:('simple-shuffle-combine-sort_index-head-partial-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 18) +[GET] object_id:fb19c57050f44530ffffffffffffffffffffffff0100000001000000 size:0 func:('head-19-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:53f9043113c8b098ffffffffffffffffffffffff0100000001000000 size:0 func:('head-19-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:03a67a370b3493f5ffffffffffffffffffffffff0100000001000000 size:0 func:('head-19-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:ee03d802e71ec0eeffffffffffffffffffffffff0100000001000000 size:0 func:('head-19-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:3bc7829716533f00ffffffffffffffffffffffff0100000001000000 size:0 func:('head-19-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:dddeeeaeeeebfe7dffffffffffffffffffffffff0100000001000000 size:0 func:('head-19-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:a6b00a3bd59cc9d6ffffffffffffffffffffffff0100000001000000 size:0 func:('head-19-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:5bf3639a3854aeb2ffffffffffffffffffffffff0100000001000000 size:0 func:('head-19-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:0f7739530230c98cffffffffffffffffffffffff0100000001000000 size:0 func:('head-19-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:8fe0d6845cd7f25dffffffffffffffffffffffff0100000001000000 size:0 func:('head-19-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:d705a284eb402aadffffffffffffffffffffffff0100000001000000 size:0 func:('head-19-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:8404c7ffc4e5df72ffffffffffffffffffffffff0100000001000000 size:0 func:('head-19-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:9aa91134e9a97a17ffffffffffffffffffffffff0100000001000000 size:0 func:('head-19-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:7f0166b85ffd7f1fffffffffffffffffffffffff0100000001000000 size:0 func:('head-19-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:31ef6b1b6ec46408ffffffffffffffffffffffff0100000001000000 size:0 func:('head-19-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:5b4f04929fdd0816ffffffffffffffffffffffff0100000001000000 size:0 func:('head-19-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:8028a08212150b3dffffffffffffffffffffffff0100000001000000 size:0 func:('head-19-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:587d36ff2f25987cffffffffffffffffffffffff0100000001000000 size:0 func:('head-19-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +[GET] object_id:e8e0cbdf2460f7b2ffffffffffffffffffffffff0100000001000000 size:0 func:('head-19-10-sort_index-ca66ea8c0355bd19d0e2a3558a5eb822', 0) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20959) +(pid=20959) +(pid=20959) +(pid=20959) +(pid=20959) +(pid=20959) +(pid=20959) +(pid=20959) +(pid=20959) +(pid=20959) +(pid=20959) +(pid=20959) +(pid=20959) +(pid=20953) +(pid=20953) +(pid=20959) +(pid=20959) +(pid=20953) +(pid=20953) +(pid=20959) +(pid=20959) +(pid=20959) +(pid=20959) +(pid=20959) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20989) +(pid=20989) +(pid=20989) +(pid=20989) +(pid=20989) +(pid=20989) +(pid=20989) +(pid=20989) +(pid=20989) +(pid=20989) +(pid=20989) +(pid=20989) +(pid=20989) +(pid=20989) +(pid=20989) +(pid=20989) +(pid=20989) +(pid=20989) +(pid=20989) +(pid=20961) +(pid=20961) +(pid=20961) +(pid=20961) +(pid=20961) +(pid=20961) +(pid=20961) +(pid=20961) +(pid=20961) +(pid=20961) +(pid=20961) +(pid=20961) +(pid=20961) +(pid=20961) +(pid=20961) +(pid=20961) +(pid=20961) +(pid=20961) +(pid=20961) +(pid=20960) +(pid=20960) +(pid=20960) +(pid=20960) +(pid=20960) +(pid=20960) +(pid=20960) +(pid=20960) +(pid=20960) +(pid=20960) +(pid=20960) +(pid=20960) +(pid=20960) +(pid=20960) +(pid=20960) +(pid=20960) +(pid=20960) +(pid=20960) +(pid=20960) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20953) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20956) +(pid=20956) +(pid=20956) +(pid=20956) +(pid=20956) +(pid=20956) +(pid=20956) +(pid=20956) +(pid=20956) +(pid=20956) +(pid=20956) +(pid=20956) +(pid=20961) +(pid=20961) +(pid=20956) +(pid=20961) +(pid=20956) +(pid=20961) +(pid=20961) +(pid=20961) +(pid=20956) +(pid=20956) +(pid=21013) +(pid=21013) +(pid=21013) +(pid=20961) +(pid=20961) +(pid=20956) +(pid=20956) +(pid=21013) +(pid=21013) +(pid=20961) +(pid=20961) +(pid=20956) +(pid=21013) +(pid=21013) +(pid=20961) +(pid=20961) +(pid=21013) +(pid=21013) +(pid=20961) +(pid=20961) +(pid=20961) +(pid=21013) +(pid=21013) +(pid=20961) +(pid=20961) +(pid=21013) +(pid=21013) +(pid=20961) +(pid=21013) +(pid=21013) +(pid=21013) +(pid=20961) +(pid=21013) +(pid=21013) +(pid=21013) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20955) +(pid=20955) +(pid=20955) +(pid=20955) +(pid=20955) +(pid=20955) +(pid=20955) +(pid=20955) +(pid=20955) +(pid=20955) +(pid=20955) +(pid=20955) +(pid=20955) +(pid=20955) +(pid=20955) +(pid=20955) +(pid=20955) +(pid=20955) +(pid=20955) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=20954) +(pid=21013) +(pid=21013) +(pid=21013) +(pid=21013) +(pid=21013) +(pid=21013) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=21013) +(pid=21013) +(pid=21013) +(pid=21013) +(pid=21013) +(pid=21013) +(pid=21013) +(pid=21013) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=21013) +(pid=21013) +(pid=20951) +(pid=20951) +(pid=21013) +(pid=21013) +(pid=20951) +(pid=21013) +(pid=21013) +(pid=21013) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20957) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +(pid=20951) +Trial 0 done after 76.37009787559509 +num obj ref bytes used: 4577.931343078613 +num objs: 400 +ray mean over 1 trials: 76.37009787559509 +- 0.0 diff --git a/release/raydp_tests/dask_on_ray/test_dask.py b/release/raydp_tests/dask_on_ray/test_dask.py index b17415b4846a..7ab8648f3cf2 100644 --- a/release/raydp_tests/dask_on_ray/test_dask.py +++ b/release/raydp_tests/dask_on_ray/test_dask.py @@ -9,16 +9,18 @@ import os.path import csv import fastparquet +import sys from dask.distributed import Client from dask.distributed import wait import cProfile, pstats, io from pstats import SortKey +from tqdm import tqdm import time -# DATA_DIR = "~/dask-on-ray-data" -DATA_DIR = "/obj-data" +DATA_DIR = "~/dask-on-ray-data" +# DATA_DIR = "/obj-data" def load_dataset(nbytes, npartitions, sort): @@ -30,11 +32,11 @@ def foo(i): filename = "df-{}-{}-{}.parquet.gzip".format( "sort" if sort else "groupby", num_bytes_per_partition, i) filename = os.path.join(DATA_DIR, filename) - print("Partition file", filename) + # print("Partition file", filename) if not os.path.exists(filename): if sort: nrows = num_bytes_per_partition // 8 - print("Allocating dataset with {} rows".format(nrows)) + # print("Allocating dataset with {} rows".format(nrows)) dataset = pd.DataFrame( np.random.randint( 0, @@ -44,34 +46,87 @@ def foo(i): columns=['a']) else: nrows = num_bytes_per_partition // (8 * 2) - print("Allocating dataset with {} rows".format(nrows)) + # print("Allocating dataset with {} rows".format(nrows)) dataset = pd.DataFrame( np.random.randint(0, 100, size=(nrows, 2), dtype=np.int64), columns=['a', 'b']) - print("Done allocating") + # print("Done allocating") dataset.to_parquet(filename, compression='gzip') - print("Done writing to disk") + # print("Done writing to disk") return filename for i in range(npartitions): filenames.append(foo.remote(i)) - filenames = ray.get(filenames) + results = [] + pbar = tqdm(total=len(filenames)) + ready, unready = ray.wait(filenames) + while unready: + results.extend(ready) + ready, unready = ray.wait(unready) + pbar.update(len(ready)) + del filenames + filenames = ray.get(results) df = dd.read_parquet(filenames) - import time - print("parquet read") - time.sleep(20) - print("sleep done.") return df def trial(nbytes, n_partitions, sort, generate_only): + from ray.util.dask import RayDaskCallback + + @ray.remote + class CounterActor: + def __init__(self): + self.cnt = 0 + self.object_num = 0 + def increment(self, n, object_num): + self.cnt += n + self.object_num += object_num + def get(self): + return self.cnt + def get_object_num(self): + return self.object_num + + c = CounterActor.remote() + + class CountRefs(RayDaskCallback): + def __init__(self, cnt_actor): + self.cnt_actor = cnt_actor + + def _ray_pretask(self, key, consumed_objs): + total = 0 + object_num = 0 + for obj in consumed_objs: + if sys.getsizeof(obj) > 1024 * 100: + print(type(obj)) + object_num += 1 + total += sys.getsizeof(obj) + self.cnt_actor.increment.remote(total, object_num) + + def _ray_postsubmit_all(self, object_refs, dsk): + pass + # print(f"dask DAG: {dsk.dicts}") + + def _ray_finish(self, result): + if isinstance(result, (list, tuple)): + result = [result] + total = 0 + object_num = 0 + for obj in result: + if sys.getsizeof(obj) > 1024 * 100: + object_num += 1 + total += sys.getsizeof(obj) + self.cnt_actor.increment.remote(total, object_num) + + + count = CountRefs(c) df = load_dataset(nbytes, n_partitions, sort) - pr = cProfile.Profile() + # pr = cProfile.Profile() if generate_only: return times = [] + # df.visualize(filename=f'a-{i}.svg') start = time.time() for i in range(1): print("Trial {} start".format(i)) @@ -79,9 +134,11 @@ def trial(nbytes, n_partitions, sort, generate_only): if sort: # pr.enable() - a = df.set_index('a', shuffle='tasks', max_branch=10**9) - # a.visualize(filename=f'a-{i}.svg') - a.head(10, npartitions=-1) + with count: + a = df.set_index('a', shuffle='tasks', max_branch=10 ** 9) + a = a.head(10, npartitions=-1, compute=False) + # a.visualize(filename=f'a-{i}.svg') + a.compute() # pr.disable() else: df.groupby('b').a.mean().compute() @@ -90,6 +147,8 @@ def trial(nbytes, n_partitions, sort, generate_only): duration = trial_end - trial_start times.append(duration) print("Trial {} done after {}".format(i, duration)) + print(f"num obj ref bytes used: {ray.get(c.get.remote()) / 1024 / 1024}") + print(f"num objs: {ray.get(c.get_object_num.remote())}") # sortby = SortKey.CUMULATIVE # ps = pstats.Stats(pr).sort_stats(sortby) # ps.dump_stats("ray_profile_data") @@ -156,7 +215,7 @@ def trial(nbytes, n_partitions, sort, generate_only): system = "dask" if args.dask else "ray" # print(system, trial(1000, 10, args.sort, args.generate_only)) - print("WARMUP DONE") + # print("WARMUP DONE") npartitions = args.npartitions if args.nbytes // npartitions > args.max_partition_size: diff --git a/src/ray/object_manager/plasma/store.cc b/src/ray/object_manager/plasma/store.cc index 9627c9dc3682..0c8eeebd1de1 100644 --- a/src/ray/object_manager/plasma/store.cc +++ b/src/ray/object_manager/plasma/store.cc @@ -398,6 +398,7 @@ void PlasmaStore::ReturnFromGet(GetRequest *get_req) { store_fds.push_back(fd); mmap_sizes.push_back(GetMmapSize(fd)); if (get_req->is_from_worker) { + RAY_LOG(ERROR) << "[GET] object_id:" << object_id << " size:" << (object.data_size + object.metadata_size); total_referenced_bytes_ += object.data_size + object.metadata_size; } }