cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Yuji Ito (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-13530) GroupCommitLogService
Date Sat, 02 Sep 2017 14:19:02 GMT

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

Yuji Ito commented on CASSANDRA-13530:
--------------------------------------

{quote}I don't think that should hurt performance, but I would really like to see testing
with it at the default value.{quote}
Sorry, I thought 2ms was the default value because the yaml file had the value. Could you
tell me the default value?
Ref. https://docs.datastax.com/en/cassandra/3.0/cassandra/configuration/configCassandra_yaml.html

{quote}Is the issue that the device is shared between data and the commit log so it's better
to have fewer larger syncs?{quote}
No. The environment has 2 storages for the data disk and the commit log disk.

{quote}Have you added a warmup phase to the testing so that everything is warmed up before
you start measuring?{quote}
Yes. I tried as below.
# Stop all C* nodes
# Remove C* data and commitlog on all nodes
# Restart all C* nodes
# Insert 10000 records
# Change the fixed throughput of my test
# Start my test
  test-warmup: Requests 2560000 update operations
  test-measure: Requests 2560000 update operations and log each latency
# Repeat from 1.

{quote}how many operations are in each batch when batch and group commitlog are syncing?{quote}
You have a point. I'll measure it.

> GroupCommitLogService
> ---------------------
>
>                 Key: CASSANDRA-13530
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-13530
>             Project: Cassandra
>          Issue Type: Improvement
>            Reporter: Yuji Ito
>            Assignee: Yuji Ito
>             Fix For: 2.2.x, 3.0.x, 3.11.x
>
>         Attachments: groupCommit22.patch, groupCommit30.patch, groupCommit3x.patch, groupCommitLog_noSerial_result.xlsx,
groupCommitLog_result.xlsx, GuavaRequestThread.java, MicroRequestThread.java
>
>
> I propose a new CommitLogService, GroupCommitLogService, to improve the throughput when
lots of requests are received.
> It improved the throughput by maximum 94%.
> I'd like to discuss about this CommitLogService.
> Currently, we can select either 2 CommitLog services; Periodic and Batch.
> In Periodic, we might lose some commit log which hasn't written to the disk.
> In Batch, we can write commit log to the disk every time. The size of commit log to write
is too small (< 4KB). When high concurrency, these writes are gathered and persisted to
the disk at once. But, when insufficient concurrency, many small writes are issued and the
performance decreases due to the latency of the disk. Even if you use SSD, processes of many
IO commands decrease the performance.
> GroupCommitLogService writes some commitlog to the disk at once.
> The patch adds GroupCommitLogService (It is enabled by setting `commitlog_sync` and `commitlog_sync_group_window_in_ms`
in cassandra.yaml).
> The difference from Batch is just only waiting for the semaphore.
> By waiting for the semaphore, some writes for commit logs are executed at the same time.
> In GroupCommitLogService, the latency becomes worse if the there is no concurrency.
> I measured the performance with my microbench (MicroRequestThread.java) by increasing
the number of threads.The cluster has 3 nodes (Replication factor: 3). Each nodes is AWS EC2
m4.large instance + 200IOPS io1 volume.
> The result is as below. The GroupCommitLogService with 10ms window improved update with
Paxos by 94% and improved select with Paxos by 76%.
> h6. SELECT / sec
> ||\# of threads||Batch 2ms||Group 10ms||
> |1|192|103|
> |2|163|212|
> |4|264|416|
> |8|454|800|
> |16|744|1311|
> |32|1151|1481|
> |64|1767|1844|
> |128|2949|3011|
> |256|4723|5000|
> h6. UPDATE / sec
> ||\# of threads||Batch 2ms||Group 10ms||
> |1|45|26|
> |2|39|51|
> |4|58|102|
> |8|102|198|
> |16|167|213|
> |32|289|295|
> |64|544|548|
> |128|1046|1058|
> |256|2020|2061|



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


Mime
View raw message