[ 
https://issues.apache.org/jira/browse/CASSANDRA-8192?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14188858#comment-14188858
 ] 

Joshua McKenzie commented on CASSANDRA-8192:
--------------------------------------------

The portion of your system.log I'm looking for is specifically right after 
initialization where the JVM params are listed - something similar to:
{noformat}
INFO  [main] 2014-10-29 13:47:13,766 CassandraDaemon.java:102 - Hostname: 
WIN-UGJOT801054
INFO  [main] 2014-10-29 13:47:13,782 YamlConfigurationLoader.java:80 - Loading 
settings from file:/C:/Users/jmckenzie/Desktop/cassandra/conf/cassandra.yaml
INFO  [main] 2014-10-29 13:47:13,860 YamlConfigurationLoader.java:123 - Node 
configuration:[authenticator=AllowAllAuthenticator; 
authorizer=AllowAllAuthorizer; auto_snapshot=true; 
batch_size_warn_threshold_in_kb=5; batchlog_replay_throttle_in_kb=1024; 
cas_contention_timeout_in_ms=1000; client_encryption_options=<REDACTED>; 
cluster_name=Test Cluster; column_index_size_in_kb=64; 
commit_failure_policy=stop; commitlog_segment_size_in_mb=32; 
commitlog_sync=periodic; commitlog_sync_period_in_ms=10000; 
compaction_throughput_mb_per_sec=16; concurrent_counter_writes=32; 
concurrent_reads=32; concurrent_writes=32; counter_cache_save_period=7200; 
counter_cache_size_in_mb=null; counter_write_request_timeout_in_ms=5000; 
cross_node_timeout=false; disk_failure_policy=stop; 
dynamic_snitch_badness_threshold=0.1; 
dynamic_snitch_reset_interval_in_ms=600000; 
dynamic_snitch_update_interval_in_ms=100; endpoint_snitch=SimpleSnitch; 
hinted_handoff_enabled=true; hinted_handoff_throttle_in_kb=1024; 
incremental_backups=false; index_summary_capacity_in_mb=null; 
index_summary_resize_interval_in_minutes=60; inter_dc_tcp_nodelay=false; 
internode_compression=all; key_cache_save_period=14400; 
key_cache_size_in_mb=null; listen_address=localhost; 
max_hint_window_in_ms=10800000; max_hints_delivery_threads=2; 
memtable_allocation_type=heap_buffers; native_transport_port=9042; 
num_tokens=256; partitioner=org.apache.cassandra.dht.Murmur3Partitioner; 
permissions_validity_in_ms=2000; range_request_timeout_in_ms=10000; 
read_request_timeout_in_ms=5000; 
request_scheduler=org.apache.cassandra.scheduler.NoScheduler; 
request_timeout_in_ms=10000; row_cache_save_period=0; row_cache_size_in_mb=0; 
rpc_address=localhost; rpc_keepalive=true; rpc_port=9160; rpc_server_type=sync; 
seed_provider=[{class_name=org.apache.cassandra.locator.SimpleSeedProvider, 
parameters=[{seeds=127.0.0.1}]}]; server_encryption_options=<REDACTED>; 
snapshot_before_compaction=false; ssl_storage_port=7001; 
sstable_preemptive_open_interval_in_mb=50; start_native_transport=true; 
start_rpc=true; storage_port=7000; thrift_framed_transport_size_in_mb=15; 
tombstone_failure_threshold=100000; tombstone_warn_threshold=1000; 
trickle_fsync=false; trickle_fsync_interval_in_kb=10240; 
truncate_request_timeout_in_ms=60000; write_request_timeout_in_ms=2000]
INFO  [main] 2014-10-29 13:47:14,016 DatabaseDescriptor.java:198 - 
DiskAccessMode 'auto' determined to be standard, indexAccessMode is standard
INFO  [main] 2014-10-29 13:47:14,031 DatabaseDescriptor.java:286 - Global 
memtable on-heap threshold is enabled at 249MB
INFO  [main] 2014-10-29 13:47:14,031 DatabaseDescriptor.java:290 - Global 
memtable off-heap threshold is enabled at 249MB
INFO  [main] 2014-10-29 13:47:14,328 YamlConfigurationLoader.java:80 - Loading 
settings from file:/C:/Users/jmckenzie/Desktop/cassandra/conf/cassandra.yaml
INFO  [main] 2014-10-29 13:47:14,343 YamlConfigurationLoader.java:123 - Node 
configuration:[authenticator=AllowAllAuthenticator; 
authorizer=AllowAllAuthorizer; auto_snapshot=true; 
batch_size_warn_threshold_in_kb=5; batchlog_replay_throttle_in_kb=1024; 
cas_contention_timeout_in_ms=1000; client_encryption_options=<REDACTED>; 
cluster_name=Test Cluster; column_index_size_in_kb=64; 
commit_failure_policy=stop; commitlog_segment_size_in_mb=32; 
commitlog_sync=periodic; commitlog_sync_period_in_ms=10000; 
compaction_throughput_mb_per_sec=16; concurrent_counter_writes=32; 
concurrent_reads=32; concurrent_writes=32; counter_cache_save_period=7200; 
counter_cache_size_in_mb=null; counter_write_request_timeout_in_ms=5000; 
cross_node_timeout=false; disk_failure_policy=stop; 
dynamic_snitch_badness_threshold=0.1; 
dynamic_snitch_reset_interval_in_ms=600000; 
dynamic_snitch_update_interval_in_ms=100; endpoint_snitch=SimpleSnitch; 
hinted_handoff_enabled=true; hinted_handoff_throttle_in_kb=1024; 
incremental_backups=false; index_summary_capacity_in_mb=null; 
index_summary_resize_interval_in_minutes=60; inter_dc_tcp_nodelay=false; 
internode_compression=all; key_cache_save_period=14400; 
key_cache_size_in_mb=null; listen_address=localhost; 
max_hint_window_in_ms=10800000; max_hints_delivery_threads=2; 
memtable_allocation_type=heap_buffers; native_transport_port=9042; 
num_tokens=256; partitioner=org.apache.cassandra.dht.Murmur3Partitioner; 
permissions_validity_in_ms=2000; range_request_timeout_in_ms=10000; 
read_request_timeout_in_ms=5000; 
request_scheduler=org.apache.cassandra.scheduler.NoScheduler; 
request_timeout_in_ms=10000; row_cache_save_period=0; row_cache_size_in_mb=0; 
rpc_address=localhost; rpc_keepalive=true; rpc_port=9160; rpc_server_type=sync; 
seed_provider=[{class_name=org.apache.cassandra.locator.SimpleSeedProvider, 
parameters=[{seeds=127.0.0.1}]}]; server_encryption_options=<REDACTED>; 
snapshot_before_compaction=false; ssl_storage_port=7001; 
sstable_preemptive_open_interval_in_mb=50; start_native_transport=true; 
start_rpc=true; storage_port=7000; thrift_framed_transport_size_in_mb=15; 
tombstone_failure_threshold=100000; tombstone_warn_threshold=1000; 
trickle_fsync=false; trickle_fsync_interval_in_kb=10240; 
truncate_request_timeout_in_ms=60000; write_request_timeout_in_ms=2000]
INFO  [main] 2014-10-29 13:47:14,359 YamlConfigurationLoader.java:80 - Loading 
settings from file:/C:/Users/jmckenzie/Desktop/cassandra/conf/cassandra.yaml
INFO  [main] 2014-10-29 13:47:14,359 YamlConfigurationLoader.java:123 - Node 
configuration:[authenticator=AllowAllAuthenticator; 
authorizer=AllowAllAuthorizer; auto_snapshot=true; 
batch_size_warn_threshold_in_kb=5; batchlog_replay_throttle_in_kb=1024; 
cas_contention_timeout_in_ms=1000; client_encryption_options=<REDACTED>; 
cluster_name=Test Cluster; column_index_size_in_kb=64; 
commit_failure_policy=stop; commitlog_segment_size_in_mb=32; 
commitlog_sync=periodic; commitlog_sync_period_in_ms=10000; 
compaction_throughput_mb_per_sec=16; concurrent_counter_writes=32; 
concurrent_reads=32; concurrent_writes=32; counter_cache_save_period=7200; 
counter_cache_size_in_mb=null; counter_write_request_timeout_in_ms=5000; 
cross_node_timeout=false; disk_failure_policy=stop; 
dynamic_snitch_badness_threshold=0.1; 
dynamic_snitch_reset_interval_in_ms=600000; 
dynamic_snitch_update_interval_in_ms=100; endpoint_snitch=SimpleSnitch; 
hinted_handoff_enabled=true; hinted_handoff_throttle_in_kb=1024; 
incremental_backups=false; index_summary_capacity_in_mb=null; 
index_summary_resize_interval_in_minutes=60; inter_dc_tcp_nodelay=false; 
internode_compression=all; key_cache_save_period=14400; 
key_cache_size_in_mb=null; listen_address=localhost; 
max_hint_window_in_ms=10800000; max_hints_delivery_threads=2; 
memtable_allocation_type=heap_buffers; native_transport_port=9042; 
num_tokens=256; partitioner=org.apache.cassandra.dht.Murmur3Partitioner; 
permissions_validity_in_ms=2000; range_request_timeout_in_ms=10000; 
read_request_timeout_in_ms=5000; 
request_scheduler=org.apache.cassandra.scheduler.NoScheduler; 
request_timeout_in_ms=10000; row_cache_save_period=0; row_cache_size_in_mb=0; 
rpc_address=localhost; rpc_keepalive=true; rpc_port=9160; rpc_server_type=sync; 
seed_provider=[{class_name=org.apache.cassandra.locator.SimpleSeedProvider, 
parameters=[{seeds=127.0.0.1}]}]; server_encryption_options=<REDACTED>; 
snapshot_before_compaction=false; ssl_storage_port=7001; 
sstable_preemptive_open_interval_in_mb=50; start_native_transport=true; 
start_rpc=true; storage_port=7000; thrift_framed_transport_size_in_mb=15; 
tombstone_failure_threshold=100000; tombstone_warn_threshold=1000; 
trickle_fsync=false; trickle_fsync_interval_in_kb=10240; 
truncate_request_timeout_in_ms=60000; write_request_timeout_in_ms=2000]
INFO  [main] 2014-10-29 13:47:14,359 CassandraDaemon.java:110 - 32bit JVM 
detected.  It is recommended to run Cassandra on a 64bit JVM for better 
performance.
INFO  [main] 2014-10-29 13:47:14,359 CassandraDaemon.java:113 - JVM 
vendor/version: Java HotSpot(TM) Client VM/1.7.0_67
INFO  [main] 2014-10-29 13:47:14,375 CassandraDaemon.java:141 - Heap size: 
1046937600/1046937600
INFO  [main] 2014-10-29 13:47:14,375 CassandraDaemon.java:143 - Code Cache 
Non-heap memory: init = 163840(160K) used = 1219904(1191K) committed = 
1245184(1216K) max = 33554432(32768K)
INFO  [main] 2014-10-29 13:47:14,375 CassandraDaemon.java:143 - Par Eden Space 
Heap memory: init = 214827008(209792K) used = 94590616(92373K) committed = 
214827008(209792K) max = 214827008(209792K)
INFO  [main] 2014-10-29 13:47:14,375 CassandraDaemon.java:143 - Par Survivor 
Space Heap memory: init = 26804224(26176K) used = 0(0K) committed = 
26804224(26176K) max = 26804224(26176K)
INFO  [main] 2014-10-29 13:47:14,375 CassandraDaemon.java:143 - CMS Old Gen 
Heap memory: init = 805306368(786432K) used = 0(0K) committed = 
805306368(786432K) max = 805306368(786432K)
INFO  [main] 2014-10-29 13:47:14,375 CassandraDaemon.java:143 - CMS Perm Gen 
Non-heap memory: init = 12582912(12288K) used = 11161888(10900K) committed = 
12582912(12288K) max = 67108864(65536K)
{noformat}

I'm unable to reproduce on a 32-bit win7 vm locally, including w/upgrade from 
2.1.0 to 2.1.1.  A complete system.log from a node exhibiting this error might 
help determine if this is a code or environment issue.

Regarding your environment: this is without powershell permissions w/legacy 
settings correct?  You have environment listed as 3G RAM however a 32-bit JVM 
won't fly with that - what do you have for your JVM params in your 
cassandra.bat?

> AssertionError in Memory.java
> -----------------------------
>
>                 Key: CASSANDRA-8192
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-8192
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>         Environment: Windows-7-32 bit, 3GB RAM, Java 1.7.0_67
>            Reporter: Andreas Schnitzerling
>            Assignee: Joshua McKenzie
>         Attachments: cassandra.yaml, system.log
>
>
> Since update of 1 of 12 nodes from 2.1.0-rel to 2.1.1-rel Exception during 
> start up.
> {panel:title=system.log}
> ERROR [SSTableBatchOpen:1] 2014-10-27 09:44:00,079 CassandraDaemon.java:153 - 
> Exception in thread Thread[SSTableBatchOpen:1,5,main]
> java.lang.AssertionError: null
>       at org.apache.cassandra.io.util.Memory.size(Memory.java:307) 
> ~[apache-cassandra-2.1.1.jar:2.1.1]
>       at 
> org.apache.cassandra.io.compress.CompressionMetadata.<init>(CompressionMetadata.java:135)
>  ~[apache-cassandra-2.1.1.jar:2.1.1]
>       at 
> org.apache.cassandra.io.compress.CompressionMetadata.create(CompressionMetadata.java:83)
>  ~[apache-cassandra-2.1.1.jar:2.1.1]
>       at 
> org.apache.cassandra.io.util.CompressedSegmentedFile$Builder.metadata(CompressedSegmentedFile.java:50)
>  ~[apache-cassandra-2.1.1.jar:2.1.1]
>       at 
> org.apache.cassandra.io.util.CompressedPoolingSegmentedFile$Builder.complete(CompressedPoolingSegmentedFile.java:48)
>  ~[apache-cassandra-2.1.1.jar:2.1.1]
>       at 
> org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:766) 
> ~[apache-cassandra-2.1.1.jar:2.1.1]
>       at 
> org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:725) 
> ~[apache-cassandra-2.1.1.jar:2.1.1]
>       at 
> org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:402) 
> ~[apache-cassandra-2.1.1.jar:2.1.1]
>       at 
> org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:302) 
> ~[apache-cassandra-2.1.1.jar:2.1.1]
>       at 
> org.apache.cassandra.io.sstable.SSTableReader$4.run(SSTableReader.java:438) 
> ~[apache-cassandra-2.1.1.jar:2.1.1]
>       at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source) 
> ~[na:1.7.0_55]
>       at java.util.concurrent.FutureTask.run(Unknown Source) ~[na:1.7.0_55]
>       at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source) 
> [na:1.7.0_55]
>       at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) 
> [na:1.7.0_55]
>       at java.lang.Thread.run(Unknown Source) [na:1.7.0_55]
> {panel}
> In the attached log you can still see as well CASSANDRA-8069 and 
> CASSANDRA-6283.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to