Maxwell-Guo commented on code in PR #2777: URL: https://github.com/apache/cassandra/pull/2777#discussion_r1360494261
########## src/java/org/apache/cassandra/db/commitlog/DirectIOSegment.java: ########## @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.db.commitlog; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.file.Files; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.io.FSWriteError; +import org.apache.cassandra.io.util.FileUtils; + +/* + * Direct-IO segment. Allocates ByteBuffer using ByteBuffer.allocateDirect and align + * ByteBuffer.position, ByteBuffer.limit and FileChannel.position to page size (4K). + * Java-11 forces minimum page size to be written to disk with Direct-IO. + */ +public class DirectIOSegment extends CommitLogSegment +{ + ByteBuffer original; + static int minimumAllowedAlign; + + // Needed to track number of bytes written to disk in multiple of page size. + int lastWritten = 0; + + /** + * Constructs a new segment file. + * + * @param commitLog the commit log it will be used with. + */ + DirectIOSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager) + { + super(commitLog, manager); + + // mark the initial sync marker as uninitialised + int firstSync = buffer.position(); + buffer.putInt(firstSync + 0, 0); + buffer.putInt(firstSync + 4, 0); + + // Testing shows writing initial bytes takes some time. During peak load, it helps + // lot and Syncer thread can actually do flush activity. Making this initial + // slow operation to be executed by Allocator thread here. + int oldLastSyncedOffset = lastSyncedOffset; + // 8 bytes are written above. + lastSyncedOffset = 8; + flush(0, lastSyncedOffset); + lastSyncedOffset = oldLastSyncedOffset; + } + + ByteBuffer createBuffer(CommitLog commitLog) + { + if (minimumAllowedAlign == 0) + { + try + { + minimumAllowedAlign = (int)Files.getFileStore(logFile.toPath()).getBlockSize(); + } + catch (IOException e) + { + throw new FSWriteError(e, logFile); + } + } + + int cl_size = DatabaseDescriptor.getCommitLogSegmentSize(); + original = ByteBuffer.allocateDirect(cl_size + minimumAllowedAlign); + + ByteBuffer alignedBuffer = original.alignedSlice(minimumAllowedAlign); + assert alignedBuffer.limit() >= cl_size : String.format("Bytebuffer slicing failed to get required buffer size (required=%d,current size=%d", cl_size, alignedBuffer.limit()); + + assert alignedBuffer.alignmentOffset(0, minimumAllowedAlign) == 0 : "Index 0 should be aligned to 4K page size"; + assert alignedBuffer.alignmentOffset(alignedBuffer.limit(), minimumAllowedAlign) == 0 : "Limit should be aligned to 4K page size" ; + + return alignedBuffer; + } + + @Override + void write(int startMarker, int nextMarker) + { + // if there's room in the discard section to write an empty header, + // zero out the next sync marker so replayer can cleanly exit + if (nextMarker <= buffer.capacity() - SYNC_MARKER_SIZE) + { + buffer.putInt(nextMarker, 0); + buffer.putInt(nextMarker + 4, 0); + } + + // write previous sync marker to point to next sync marker + // we don't chain the crcs here to ensure this method is idempotent if it fails + writeSyncMarker(id, buffer, startMarker, startMarker, nextMarker); + } + + @Override + protected void flush(int startMarker, int nextMarker) + { + + try + { + // lastSyncedOffset is synced to disk. Align lastSyncedOffset to start of 4K page + // and nextMarker to end of 4K page to avoid write errors. + long filePosition = lastSyncedOffset; + ByteBuffer duplicate = buffer.duplicate(); + long file_old_position = channel.position(); Review Comment: file_old_position is not used ? ########## src/java/org/apache/cassandra/db/commitlog/DirectIOSegment.java: ########## @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.db.commitlog; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.file.Files; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.io.FSWriteError; +import org.apache.cassandra.io.util.FileUtils; + +/* + * Direct-IO segment. Allocates ByteBuffer using ByteBuffer.allocateDirect and align + * ByteBuffer.position, ByteBuffer.limit and FileChannel.position to page size (4K). + * Java-11 forces minimum page size to be written to disk with Direct-IO. + */ +public class DirectIOSegment extends CommitLogSegment +{ + ByteBuffer original; + static int minimumAllowedAlign; + + // Needed to track number of bytes written to disk in multiple of page size. + int lastWritten = 0; + + /** + * Constructs a new segment file. + * + * @param commitLog the commit log it will be used with. + */ + DirectIOSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager) + { + super(commitLog, manager); + + // mark the initial sync marker as uninitialised + int firstSync = buffer.position(); + buffer.putInt(firstSync + 0, 0); + buffer.putInt(firstSync + 4, 0); + + // Testing shows writing initial bytes takes some time. During peak load, it helps + // lot and Syncer thread can actually do flush activity. Making this initial + // slow operation to be executed by Allocator thread here. + int oldLastSyncedOffset = lastSyncedOffset; + // 8 bytes are written above. + lastSyncedOffset = 8; + flush(0, lastSyncedOffset); + lastSyncedOffset = oldLastSyncedOffset; + } + + ByteBuffer createBuffer(CommitLog commitLog) + { + if (minimumAllowedAlign == 0) + { + try + { + minimumAllowedAlign = (int)Files.getFileStore(logFile.toPath()).getBlockSize(); + } + catch (IOException e) + { + throw new FSWriteError(e, logFile); + } + } + + int cl_size = DatabaseDescriptor.getCommitLogSegmentSize(); Review Comment: I think we can rename this to segmentSize ########## src/java/org/apache/cassandra/db/commitlog/DirectIOSegment.java: ########## @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.db.commitlog; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.file.Files; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.io.FSWriteError; +import org.apache.cassandra.io.util.FileUtils; + +/* + * Direct-IO segment. Allocates ByteBuffer using ByteBuffer.allocateDirect and align + * ByteBuffer.position, ByteBuffer.limit and FileChannel.position to page size (4K). + * Java-11 forces minimum page size to be written to disk with Direct-IO. + */ +public class DirectIOSegment extends CommitLogSegment +{ + ByteBuffer original; + static int minimumAllowedAlign; + + // Needed to track number of bytes written to disk in multiple of page size. + int lastWritten = 0; + + /** + * Constructs a new segment file. + * + * @param commitLog the commit log it will be used with. + */ + DirectIOSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager) + { + super(commitLog, manager); + + // mark the initial sync marker as uninitialised + int firstSync = buffer.position(); + buffer.putInt(firstSync + 0, 0); + buffer.putInt(firstSync + 4, 0); + + // Testing shows writing initial bytes takes some time. During peak load, it helps + // lot and Syncer thread can actually do flush activity. Making this initial + // slow operation to be executed by Allocator thread here. + int oldLastSyncedOffset = lastSyncedOffset; + // 8 bytes are written above. + lastSyncedOffset = 8; + flush(0, lastSyncedOffset); + lastSyncedOffset = oldLastSyncedOffset; + } + + ByteBuffer createBuffer(CommitLog commitLog) + { + if (minimumAllowedAlign == 0) + { + try + { + minimumAllowedAlign = (int)Files.getFileStore(logFile.toPath()).getBlockSize(); Review Comment: why you cast the long value to int ? ########## src/java/org/apache/cassandra/db/commitlog/DirectIOSegment.java: ########## @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.db.commitlog; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.file.Files; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.io.FSWriteError; +import org.apache.cassandra.io.util.FileUtils; + +/* + * Direct-IO segment. Allocates ByteBuffer using ByteBuffer.allocateDirect and align + * ByteBuffer.position, ByteBuffer.limit and FileChannel.position to page size (4K). + * Java-11 forces minimum page size to be written to disk with Direct-IO. + */ +public class DirectIOSegment extends CommitLogSegment +{ + ByteBuffer original; + static int minimumAllowedAlign; + + // Needed to track number of bytes written to disk in multiple of page size. + int lastWritten = 0; + + /** + * Constructs a new segment file. + * + * @param commitLog the commit log it will be used with. + */ + DirectIOSegment(CommitLog commitLog, AbstractCommitLogSegmentManager manager) + { + super(commitLog, manager); + + // mark the initial sync marker as uninitialised + int firstSync = buffer.position(); + buffer.putInt(firstSync + 0, 0); + buffer.putInt(firstSync + 4, 0); + + // Testing shows writing initial bytes takes some time. During peak load, it helps + // lot and Syncer thread can actually do flush activity. Making this initial + // slow operation to be executed by Allocator thread here. + int oldLastSyncedOffset = lastSyncedOffset; + // 8 bytes are written above. + lastSyncedOffset = 8; + flush(0, lastSyncedOffset); + lastSyncedOffset = oldLastSyncedOffset; + } + + ByteBuffer createBuffer(CommitLog commitLog) + { + if (minimumAllowedAlign == 0) + { + try + { + minimumAllowedAlign = (int)Files.getFileStore(logFile.toPath()).getBlockSize(); + } + catch (IOException e) + { + throw new FSWriteError(e, logFile); + } + } + + int cl_size = DatabaseDescriptor.getCommitLogSegmentSize(); + original = ByteBuffer.allocateDirect(cl_size + minimumAllowedAlign); + + ByteBuffer alignedBuffer = original.alignedSlice(minimumAllowedAlign); + assert alignedBuffer.limit() >= cl_size : String.format("Bytebuffer slicing failed to get required buffer size (required=%d,current size=%d", cl_size, alignedBuffer.limit()); + + assert alignedBuffer.alignmentOffset(0, minimumAllowedAlign) == 0 : "Index 0 should be aligned to 4K page size"; + assert alignedBuffer.alignmentOffset(alignedBuffer.limit(), minimumAllowedAlign) == 0 : "Limit should be aligned to 4K page size" ; + + return alignedBuffer; + } + + @Override + void write(int startMarker, int nextMarker) + { + // if there's room in the discard section to write an empty header, + // zero out the next sync marker so replayer can cleanly exit + if (nextMarker <= buffer.capacity() - SYNC_MARKER_SIZE) + { + buffer.putInt(nextMarker, 0); + buffer.putInt(nextMarker + 4, 0); + } + + // write previous sync marker to point to next sync marker + // we don't chain the crcs here to ensure this method is idempotent if it fails + writeSyncMarker(id, buffer, startMarker, startMarker, nextMarker); + } + + @Override + protected void flush(int startMarker, int nextMarker) + { + + try + { + // lastSyncedOffset is synced to disk. Align lastSyncedOffset to start of 4K page + // and nextMarker to end of 4K page to avoid write errors. + long filePosition = lastSyncedOffset; + ByteBuffer duplicate = buffer.duplicate(); + long file_old_position = channel.position(); + + // Aligned file position if not aligned to start of 4K page. + if (filePosition % minimumAllowedAlign != 0 ) + { + filePosition = filePosition & ~(minimumAllowedAlign -1); + channel.position(filePosition); + } + duplicate.position((int)filePosition); + + int flushSizeInBytes = nextMarker; + + // Align last byte to end of 4K page. + if (flushSizeInBytes % minimumAllowedAlign !=0) + flushSizeInBytes = (flushSizeInBytes + minimumAllowedAlign) & ~(minimumAllowedAlign -1); + + duplicate.limit(flushSizeInBytes); + + channel.write(duplicate); + + // Direct I/O always writes flushes in block size and writes more than the flush size. + // File size on disk will always multiple of page size and taking this into account + // helps testcases to pass. + if (flushSizeInBytes > lastWritten) { + manager.addSize(flushSizeInBytes - lastWritten); + lastWritten = (int)flushSizeInBytes; + } + } + catch (IOException e) + { + throw new FSWriteError(e, getPath()); + } + } + + @Override + public long onDiskSize() + { + return lastWritten ; Review Comment: Remove trailing spaces ########## src/java/org/apache/cassandra/db/commitlog/DirectIOSegment.java: ########## @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.db.commitlog; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.file.Files; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.io.FSWriteError; +import org.apache.cassandra.io.util.FileUtils; + +/* + * Direct-IO segment. Allocates ByteBuffer using ByteBuffer.allocateDirect and align + * ByteBuffer.position, ByteBuffer.limit and FileChannel.position to page size (4K). + * Java-11 forces minimum page size to be written to disk with Direct-IO. + */ +public class DirectIOSegment extends CommitLogSegment +{ + ByteBuffer original; + static int minimumAllowedAlign; + + // Needed to track number of bytes written to disk in multiple of page size. + int lastWritten = 0; Review Comment: int -> long ? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]

