Merge lp:~pbeaman/akiban-persistit/better_4hour_tpcc_2 into lp:akiban-persistit
- better_4hour_tpcc_2
- Merge into trunk
Status: | Superseded |
---|---|
Proposed branch: | lp:~pbeaman/akiban-persistit/better_4hour_tpcc_2 |
Merge into: | lp:akiban-persistit |
Prerequisite: | lp:~pbeaman/akiban-persistit/eclipse-canonical-format |
Diff against target: |
4095 lines (+1432/-609) 121 files modified
doc/Configuration.rst (+13/-1) eclipse-format.xml (+72/-0) src/main/java/com/persistit/Buffer.java (+96/-62) src/main/java/com/persistit/BufferPool.java (+144/-109) src/main/java/com/persistit/CLI.java (+1/-1) src/main/java/com/persistit/CheckpointManager.java (+2/-1) src/main/java/com/persistit/CleanupManager.java (+10/-0) src/main/java/com/persistit/Configuration.java (+72/-72) src/main/java/com/persistit/Exchange.java (+5/-0) src/main/java/com/persistit/IOMeter.java (+16/-9) src/main/java/com/persistit/JournalManager.java (+118/-56) src/main/java/com/persistit/JournalManagerBench.java (+186/-0) src/main/java/com/persistit/JournalTool.java (+5/-8) src/main/java/com/persistit/MVV.java (+33/-1) src/main/java/com/persistit/MediatedFileChannel.java (+1/-1) src/main/java/com/persistit/Persistit.java (+25/-12) src/main/java/com/persistit/Transaction.java (+10/-4) src/main/java/com/persistit/TransactionIndex.java (+1/-1) src/main/java/com/persistit/logging/LogBase.java (+9/-0) src/main/java/com/persistit/mxbeans/IOMeterMXBean.java (+6/-6) src/main/java/com/persistit/mxbeans/JournalManagerMXBean.java (+6/-0) src/test/java/com/persistit/AccumulatorMemoryTest.java (+0/-2) src/test/java/com/persistit/AccumulatorRecoveryTest.java (+0/-1) src/test/java/com/persistit/AccumulatorTest.java (+0/-1) src/test/java/com/persistit/AlertMonitorTest.java (+0/-1) src/test/java/com/persistit/BackupTaskTest.java (+0/-1) src/test/java/com/persistit/BufferMaxPack.java (+0/-1) src/test/java/com/persistit/BufferPoolTest.java (+0/-1) src/test/java/com/persistit/BufferTest.java (+0/-1) src/test/java/com/persistit/BufferTest2.java (+0/-1) src/test/java/com/persistit/Bug1003578Test.java (+0/-1) src/test/java/com/persistit/Bug1017957Test.java (+0/-1) src/test/java/com/persistit/Bug1018526Test.java (+0/-1) src/test/java/com/persistit/Bug706132Test.java (+0/-1) src/test/java/com/persistit/Bug708592Test.java (+0/-1) src/test/java/com/persistit/Bug739533Test.java (+0/-2) src/test/java/com/persistit/Bug777918Test.java (+0/-1) src/test/java/com/persistit/Bug790709Test.java (+0/-2) src/test/java/com/persistit/Bug870352Test.java (+0/-2) src/test/java/com/persistit/Bug877656Test.java (+0/-2) src/test/java/com/persistit/Bug882219Test.java (+0/-1) src/test/java/com/persistit/Bug885477Test.java (+0/-2) src/test/java/com/persistit/Bug889850Test.java (+0/-2) src/test/java/com/persistit/Bug911849Test.java (+0/-1) src/test/java/com/persistit/Bug912514Test.java (+0/-2) src/test/java/com/persistit/Bug915594Test.java (+0/-2) src/test/java/com/persistit/Bug918909Test.java (+0/-2) src/test/java/com/persistit/Bug920754Test.java (+0/-2) src/test/java/com/persistit/Bug923790Test.java (+0/-1) src/test/java/com/persistit/Bug927701Test.java (+1/-3) src/test/java/com/persistit/Bug932097Test.java (+0/-2) src/test/java/com/persistit/Bug937877Test.java (+0/-1) src/test/java/com/persistit/Bug942669Test.java (+0/-2) src/test/java/com/persistit/Bug947182Test.java (+0/-1) src/test/java/com/persistit/Bug974589Test.java (+0/-1) src/test/java/com/persistit/Bug980292Test.java (+0/-1) src/test/java/com/persistit/Bug989202Test.java (+0/-2) src/test/java/com/persistit/Bug992801Test.java (+0/-2) src/test/java/com/persistit/Bug996241Test.java (+0/-1) src/test/java/com/persistit/ClassIndexTest.java (+0/-1) src/test/java/com/persistit/CleanupManagerTest.java (+0/-1) src/test/java/com/persistit/CommandLineTest.java (+0/-1) src/test/java/com/persistit/ConfigurationTest.java (+0/-1) src/test/java/com/persistit/CorruptVolumeTest.java (+0/-1) src/test/java/com/persistit/DumpTaskTest.java (+0/-1) src/test/java/com/persistit/FastIndexTest.java (+0/-1) src/test/java/com/persistit/FatalErrorExceptionTest.java (+0/-1) src/test/java/com/persistit/IOFailureTest.java (+1/-2) src/test/java/com/persistit/IOMeterChargeBenchmark.java (+0/-2) src/test/java/com/persistit/InsertSequenceTest.java (+0/-1) src/test/java/com/persistit/IntegrityCheckTest.java (+5/-7) src/test/java/com/persistit/JournalManagerTest.java (+4/-1) src/test/java/com/persistit/KeyHistogramTest.java (+0/-2) src/test/java/com/persistit/MVCCPruneBufferTest.java (+25/-4) src/test/java/com/persistit/MVCCTestBase.java (+0/-1) src/test/java/com/persistit/PersistitUnitTestCase.java (+136/-0) src/test/java/com/persistit/RecoveryTest.java (+2/-1) src/test/java/com/persistit/SplitPolicyTest.java (+0/-1) src/test/java/com/persistit/StatisticsTaskTest.java (+0/-1) src/test/java/com/persistit/StressRunner.java (+2/-0) src/test/java/com/persistit/TrackingFileChannel.java (+175/-0) src/test/java/com/persistit/TransactionLifetimeTest.java (+4/-1) src/test/java/com/persistit/TransactionTest2.java (+0/-1) src/test/java/com/persistit/TreeLifetimeTest.java (+0/-1) src/test/java/com/persistit/TreeTest2.java (+0/-2) src/test/java/com/persistit/ValueTest7.java (+0/-1) src/test/java/com/persistit/VolumeTest.java (+0/-1) src/test/java/com/persistit/WarmupTest.java (+84/-31) src/test/java/com/persistit/stress/PreloadMixtureTxn1.java (+82/-0) src/test/java/com/persistit/stress/unit/CommitBench.java (+51/-7) src/test/java/com/persistit/stress/unit/TestSamePageOptimization.java (+1/-1) src/test/java/com/persistit/unit/BufferPoolMemConfigurationTest.java (+1/-0) src/test/java/com/persistit/unit/DefaultCoderManagerTest.java (+1/-0) src/test/java/com/persistit/unit/ExchangeTest.java (+1/-0) src/test/java/com/persistit/unit/FileLockTest.java (+1/-0) src/test/java/com/persistit/unit/JoinTest1.java (+1/-0) src/test/java/com/persistit/unit/KeyCoderTest1.java (+1/-0) src/test/java/com/persistit/unit/KeyFilterTest1.java (+1/-0) src/test/java/com/persistit/unit/KeyFilterTest2.java (+1/-0) src/test/java/com/persistit/unit/KeyParserTest1.java (+1/-0) src/test/java/com/persistit/unit/KeyParserTest2.java (+1/-0) src/test/java/com/persistit/unit/KeyTest1.java (+1/-0) src/test/java/com/persistit/unit/Log4JLoggerTest.java (+1/-0) src/test/java/com/persistit/unit/LongRecordTest1.java (+1/-0) src/test/java/com/persistit/unit/LotsaSmallKeys.java (+1/-0) src/test/java/com/persistit/unit/PersistitMapTest.java (+1/-0) src/test/java/com/persistit/unit/PersistitUnitTestCase.java (+0/-132) src/test/java/com/persistit/unit/SaveLoadTest1.java (+1/-0) src/test/java/com/persistit/unit/SimpleTest1.java (+1/-0) src/test/java/com/persistit/unit/TemporaryVolumeTest1.java (+1/-0) src/test/java/com/persistit/unit/TransactionTest1.java (+1/-0) src/test/java/com/persistit/unit/TransactionTest3.java (+1/-0) src/test/java/com/persistit/unit/TreeTest1.java (+1/-0) src/test/java/com/persistit/unit/UnitTestProperties.java (+0/-1) src/test/java/com/persistit/unit/ValueCoderTest1.java (+1/-0) src/test/java/com/persistit/unit/ValueCoderTest2.java (+1/-0) src/test/java/com/persistit/unit/ValueTest1.java (+1/-0) src/test/java/com/persistit/unit/ValueTest2.java (+1/-0) src/test/java/com/persistit/unit/ValueTest3.java (+1/-0) src/test/java/com/persistit/unit/ValueTest4.java (+1/-0) src/test/java/com/persistit/unit/ValueTest5.java (+1/-0) |
To merge this branch: | bzr merge lp:~pbeaman/akiban-persistit/better_4hour_tpcc_2 |
Related bugs: |
Reviewer | Review Type | Date Requested | Status |
---|---|---|---|
Akiban Technologies | Pending | ||
Review via email: mp+121189@code.launchpad.net |
This proposal supersedes a proposal from 2012-08-15.
This proposal has been superseded by a proposal from 2012-08-24.
Commit message
Description of the change
This branch solves several problems relating to Journal I/O, pruning and performance.
Specifically, when running the 4-hour TPCC test on a particular server, perf02, the "terminal" threads outrun the ability of the JOURNAL_COPIER thread to remove obsolete journal files. The result is an ever-growing collection of journal files that eventually overflow the disk. The distinguishing feature of perf02 is that it has 12 fast cores (x2 with hyper-threading) but just a single disk drive.
This branch keeps the number of journal files under control during the test (never exceeding 16) and allows the test to complete. The test arrives at a stable configuration after a couple of hours and there's no reason to believe it would fail if run for much longer periods of time. However, a longer test run has not yet been completed.
Major changes:
1) The Transaction#
2) The order in which commit() completes its actions has changed to:
flush()
notifyComple
throttle();
waitForDurab
Further, durability is measured from a timestamp taken immediately after the call to flush().
It is safe to call notifyComplete() and allow other concurrent transactions to continue (thereby hopefully allowing more transactions to join a GROUP commit). It is safe because no other transaction released by notifyComplete() can commit and become durable unless this one is also durable.
3) Pruning is accelerated, preventing old aborted transactions from holding up the JOURNAL_COPIER cleanup process. There are two main changes to support faster pruning:
3a) Scheduling: the CLEANUP_MANAGER normally polls once per second, but when the queue becomes more than half full it now runs continuously until the queue is no longer half full.
3b) Prune on write: when a dirty page thats needs pruning is being written to disk, a new optional code path prunes it before writing it. This reduces the likelihood that a page that is read back in from disk will need to be pruned.
Other changes made while analyzing test runs include updates in the IOMeter log function, and small fixes in the CLI.
One source a many lines of diff output is that to enable a new test the PersistitUnitTe
Due to the large size of this proposal, it not be approved until after Akiban Server 1.4.0 - and for that reason perhaps deserves a new version number, which is not in the current proposal.
This branch depends on the eclipse-
- 370. By Peter Beaman
-
Merge from release-notes-3.16
- 371. By Peter Beaman
-
Merge from trunk, modify according to review comments.
- 372. By Peter Beaman
-
One more review comment.
- 373. By Peter Beaman
-
Remove extraneous eclipse-format file
Unmerged revisions
Preview Diff
1 | === modified file 'doc/Configuration.rst' |
2 | --- doc/Configuration.rst 2012-06-11 21:25:37 +0000 |
3 | +++ doc/Configuration.rst 2012-08-24 14:04:53 +0000 |
4 | @@ -221,7 +221,7 @@ |
5 | ``serialOverride``, ``constructorOverride``: (``com.persistit.Configuration#setSerialOverride`` ``com.persistit.Configuration#setConstructorOverride``) |
6 | Control aspects of object serialization. See :ref:`Serialization`. |
7 | |
8 | - ``showgui``: (``com.persistit.Configuration#setShowGUI``), True of False. |
9 | + ``showgui``: (``com.persistit.Configuration#setShowGUI``), True or False (default). |
10 | If true, Persistit attempts to create and display an instance of the AdminUI utility panel within the current JVM. |
11 | Alternatively, AdminUI uses RMI and can be launched and run remotely if ``rmiport`` or ``rmihost`` has been |
12 | specified. |
13 | @@ -231,6 +231,18 @@ |
14 | install a logging adapter to reroute messages through Log4J, SLF4J or other logger. The ``logfile`` property is used |
15 | only when no adapter has been installed. |
16 | |
17 | + ``bufferinventory``: (``com.persistit.Configuration#setBufferInventoryEnabled``), True or False (default). |
18 | + If true, Persistit periodically records an inventory of all the buffes in the buffers pools to the System Volume. The inventory |
19 | + enables Persistit to preload the buffer pools then next time it starts up with approximately the same pages that were present |
20 | + before shutdown. To enable buffer preloading, the bufferpreload property must also be true. |
21 | + |
22 | + ``bufferpreload``: (``com.persistit.Configuration#setBufferPreloadEnabled``), True or False (default). |
23 | + If true, and if a buffer pool inventory was previously recorded, Persistit attempts to "warm up" the buffer pool |
24 | + by preloading pages that were present in the buffer pool when Persistit last shut down. This may allow a freshly started |
25 | + Persistit instance to begin service a workload similar to what it had previously been processing without incurring the |
26 | + cost of many random disk reads to load pages. |
27 | + |
28 | + |
29 | For all integer-valued properties, the suffix “K” may be used to represent kilo, “M” for mega, “G” for giga and “T” for tera. For example, “2M” represents the value 2,097,152. |
30 | |
31 | A Configuration Example |
32 | |
33 | === added file 'eclipse-format.xml' |
34 | --- eclipse-format.xml 1970-01-01 00:00:00 +0000 |
35 | +++ eclipse-format.xml 2012-08-24 14:04:53 +0000 |
36 | @@ -0,0 +1,72 @@ |
37 | +<!-- |
38 | + |
39 | + Copyright © 2011-2012 Akiban Technologies, Inc. All rights reserved. |
40 | + |
41 | + This program and the accompanying materials are made available |
42 | + under the terms of the Eclipse Public License v1.0 which |
43 | + accompanies this distribution, and is available at |
44 | + http://www.eclipse.org/legal/epl-v10.html |
45 | + |
46 | + This program may also be available under different license terms. |
47 | + For more information, see www.akiban.com or contact licensing@akiban.com. |
48 | + |
49 | + Contributors: |
50 | + Akiban Technologies, Inc. |
51 | + |
52 | +--> |
53 | +<?xml version="1.0" encoding="UTF-8" standalone="no"?> |
54 | +<profiles version="2"> |
55 | +<profile kind="CleanUpProfile" name="Akiban-Persistit" version="2"> |
56 | +<setting id="cleanup.format_source_code" value="true"/> |
57 | +<setting id="cleanup.add_missing_annotations" value="true"/> |
58 | +<setting id="cleanup.use_this_for_non_static_method_access_only_if_necessary" value="true"/> |
59 | +<setting id="cleanup.remove_unused_private_types" value="true"/> |
60 | +<setting id="cleanup.qualify_static_member_accesses_through_instances_with_declaring_class" value="true"/> |
61 | +<setting id="cleanup.qualify_static_method_accesses_with_declaring_class" value="false"/> |
62 | +<setting id="cleanup.add_generated_serial_version_id" value="false"/> |
63 | +<setting id="cleanup.make_variable_declarations_final" value="true"/> |
64 | +<setting id="cleanup.add_missing_methods" value="false"/> |
65 | +<setting id="cleanup.always_use_this_for_non_static_field_access" value="false"/> |
66 | +<setting id="cleanup.remove_trailing_whitespaces_ignore_empty" value="false"/> |
67 | +<setting id="cleanup.correct_indentation" value="true"/> |
68 | +<setting id="cleanup.never_use_parentheses_in_expressions" value="true"/> |
69 | +<setting id="cleanup.add_serial_version_id" value="false"/> |
70 | +<setting id="cleanup.remove_unused_private_methods" value="true"/> |
71 | +<setting id="cleanup.use_this_for_non_static_field_access" value="false"/> |
72 | +<setting id="cleanup.use_blocks_only_for_return_and_throw" value="false"/> |
73 | +<setting id="cleanup.remove_unused_private_members" value="false"/> |
74 | +<setting id="cleanup.add_missing_override_annotations_interface_methods" value="true"/> |
75 | +<setting id="cleanup.remove_trailing_whitespaces_all" value="true"/> |
76 | +<setting id="cleanup.make_type_abstract_if_missing_method" value="false"/> |
77 | +<setting id="cleanup.always_use_this_for_non_static_method_access" value="false"/> |
78 | +<setting id="cleanup.remove_unnecessary_nls_tags" value="true"/> |
79 | +<setting id="cleanup.format_source_code_changes_only" value="false"/> |
80 | +<setting id="cleanup.qualify_static_field_accesses_with_declaring_class" value="false"/> |
81 | +<setting id="cleanup.add_missing_nls_tags" value="false"/> |
82 | +<setting id="cleanup.use_this_for_non_static_field_access_only_if_necessary" value="true"/> |
83 | +<setting id="cleanup.qualify_static_member_accesses_through_subtypes_with_declaring_class" value="false"/> |
84 | +<setting id="cleanup.remove_unnecessary_casts" value="true"/> |
85 | +<setting id="cleanup.qualify_static_member_accesses_with_declaring_class" value="true"/> |
86 | +<setting id="cleanup.use_parentheses_in_expressions" value="false"/> |
87 | +<setting id="cleanup.remove_unused_private_fields" value="true"/> |
88 | +<setting id="cleanup.make_parameters_final" value="true"/> |
89 | +<setting id="cleanup.remove_trailing_whitespaces" value="true"/> |
90 | +<setting id="cleanup.remove_unused_imports" value="true"/> |
91 | +<setting id="cleanup.organize_imports" value="true"/> |
92 | +<setting id="cleanup.sort_members" value="false"/> |
93 | +<setting id="cleanup.remove_private_constructors" value="true"/> |
94 | +<setting id="cleanup.convert_to_enhanced_for_loop" value="false"/> |
95 | +<setting id="cleanup.always_use_blocks" value="true"/> |
96 | +<setting id="cleanup.never_use_blocks" value="false"/> |
97 | +<setting id="cleanup.always_use_parentheses_in_expressions" value="false"/> |
98 | +<setting id="cleanup.use_this_for_non_static_method_access" value="false"/> |
99 | +<setting id="cleanup.remove_unused_local_variables" value="false"/> |
100 | +<setting id="cleanup.make_private_fields_final" value="true"/> |
101 | +<setting id="cleanup.add_missing_deprecated_annotations" value="true"/> |
102 | +<setting id="cleanup.add_default_serial_version_id" value="true"/> |
103 | +<setting id="cleanup.sort_members_all" value="false"/> |
104 | +<setting id="cleanup.use_blocks" value="false"/> |
105 | +<setting id="cleanup.add_missing_override_annotations" value="true"/> |
106 | +<setting id="cleanup.make_local_variable_final" value="true"/> |
107 | +</profile> |
108 | +</profiles> |
109 | |
110 | === modified file 'src/main/java/com/persistit/Buffer.java' |
111 | --- src/main/java/com/persistit/Buffer.java 2012-08-24 14:04:53 +0000 |
112 | +++ src/main/java/com/persistit/Buffer.java 2012-08-24 14:04:53 +0000 |
113 | @@ -264,6 +264,9 @@ |
114 | |
115 | private final static int BINARY_SEARCH_THRESHOLD = 6; |
116 | |
117 | + private final static int PRUNE_MVV_HELPER_CHANGED = 1; |
118 | + private final static int PRUNE_MVV_HELPER_HAS_LONG = 2; |
119 | + |
120 | abstract static class VerifyVisitor { |
121 | |
122 | protected void visitPage(final long timestamp, final Volume volume, final long page, final int type, |
123 | @@ -502,15 +505,23 @@ |
124 | Debug.$assert0.t(isMine()); |
125 | final long checkpointTimestamp = _persistit.getTimestampAllocator().getProposedCheckpointTimestamp(); |
126 | if (isDirty() && !isTemporary() && getTimestamp() < checkpointTimestamp && timestamp > checkpointTimestamp) { |
127 | - writePage(); |
128 | + writePage(false); |
129 | _pool.bumpForcedCheckpointWrites(); |
130 | } |
131 | } |
132 | |
133 | void writePage() throws PersistitException { |
134 | + writePage(_persistit.getJournalManager().isWritePagePruningEnabled()); |
135 | + } |
136 | + |
137 | + private void writePage(final boolean prune) throws PersistitException { |
138 | + assert isMine(); |
139 | _persistit.checkFatal(); |
140 | final Volume volume = getVolume(); |
141 | if (volume != null) { |
142 | + if (prune) { |
143 | + pruneMvvValues(null, false); |
144 | + } |
145 | clearSlack(); |
146 | save(); |
147 | _vol.getStorage().writePage(this); |
148 | @@ -1501,9 +1512,13 @@ |
149 | Debug.$assert0.t(p + KEYBLOCK_LENGTH < _keyBlockEnd ? pointer > 0 : true); |
150 | putInt(newTail + TAILBLOCK_POINTER, pointer); |
151 | } else { |
152 | - final int storedLength = valueHelper.storeVersion(_bytes, newTail + _tailHeaderSize + klength, -1, |
153 | - _bytes.length); // TODO limit |
154 | - incCountIfMvv(_bytes, newTail + _tailHeaderSize + klength, storedLength & MVV.STORE_LENGTH_MASK); |
155 | + final int offset = newTail + _tailHeaderSize + klength; |
156 | + final int storedLength = valueHelper.storeVersion(_bytes, offset, -1, _bytes.length); // TODO |
157 | + // limit |
158 | + incCountIfMvv(_bytes, offset, storedLength & MVV.STORE_LENGTH_MASK); |
159 | + |
160 | + // TODO - remove after debugging |
161 | + MVV.verify(_persistit.getTransactionIndex(), _bytes, offset, storedLength & MVV.STORE_LENGTH_MASK); |
162 | } |
163 | // |
164 | // Correct not to call getFastIndex() |
165 | @@ -1629,9 +1644,14 @@ |
166 | Debug.$assert0.t(p + KEYBLOCK_LENGTH < _keyBlockEnd ? pointer > 0 : pointer == -1); |
167 | putInt(newTail + TAILBLOCK_POINTER, (int) pointer); |
168 | } else { |
169 | - final int storedLength = valueHelper.storeVersion(_bytes, newTail + _tailHeaderSize + klength, oldTailSize |
170 | - - _tailHeaderSize - klength, _bytes.length); // TODO - limit |
171 | - isMVV = isValueMVV(_bytes, newTail + _tailHeaderSize + klength, storedLength & MVV.STORE_LENGTH_MASK); |
172 | + final int offset = newTail + _tailHeaderSize + klength; |
173 | + final int storedLength = valueHelper.storeVersion(_bytes, offset, oldTailSize - _tailHeaderSize - klength, |
174 | + _bytes.length); // TODO - limit |
175 | + isMVV = isValueMVV(_bytes, offset, storedLength & MVV.STORE_LENGTH_MASK); |
176 | + |
177 | + // TODO - remove after debugging |
178 | + MVV.verify(_persistit.getTransactionIndex(), _bytes, offset, storedLength & MVV.STORE_LENGTH_MASK); |
179 | + |
180 | } |
181 | if (!wasMVV && isMVV) { |
182 | _mvvCount++; |
183 | @@ -3466,7 +3486,12 @@ |
184 | if (visitor != null) { |
185 | visitor.visitDataRecord(key, p, tail, klength, offset, length, getBytes()); |
186 | } |
187 | - Debug.$assert1.t(MVV.verify(_bytes, offset, length)); |
188 | + |
189 | + if (!MVV.verify(_bytes, offset, length)) { |
190 | + throw new InvalidPageStructureException("invalid MVV record at offset/length=" + offset + "/" |
191 | + + length); |
192 | + } |
193 | + |
194 | } |
195 | |
196 | if (_pool != null && getKeyCount() > _pool.getMaxKeys()) { |
197 | @@ -3546,63 +3571,15 @@ |
198 | final long timestamp = _persistit.getTimestampAllocator().updateTimestamp(); |
199 | _mvvCount = 0; |
200 | writePageOnCheckpoint(timestamp); |
201 | - final List<PrunedVersion> prunedVersions = new ArrayList<PrunedVersion>(); |
202 | - for (int p = KEY_BLOCK_START; p < _keyBlockEnd; p += KEYBLOCK_LENGTH) { |
203 | - final int kbData = getInt(p); |
204 | - final int tail = decodeKeyBlockTail(kbData); |
205 | - final int tbData = getInt(tail); |
206 | - final int klength = decodeTailBlockKLength(tbData); |
207 | - final int oldTailSize = decodeTailBlockSize(tbData); |
208 | - final int offset = tail + _tailHeaderSize + klength; |
209 | - final int oldSize = oldTailSize - klength - _tailHeaderSize; |
210 | - |
211 | - if (oldSize > 0) { |
212 | - int valueByte = _bytes[offset] & 0xFF; |
213 | - if (isLongMVV(_bytes, offset, oldSize)) { |
214 | - /* |
215 | - * Can't prune in this pass because of long record |
216 | - * timestamp management. Simply remember that there |
217 | - * are long records to prune, then prune them in a |
218 | - * copy of the buffer. |
219 | - */ |
220 | - hasLongMvvRecords = true; |
221 | - } |
222 | - if (valueByte == MVV.TYPE_MVV) { |
223 | - final int newSize = MVV.prune(_bytes, offset, oldSize, _persistit.getTransactionIndex(), |
224 | - true, prunedVersions); |
225 | - if (newSize != oldSize) { |
226 | - changed = true; |
227 | - final int newTailSize = klength + newSize + _tailHeaderSize; |
228 | - final int oldNext = (tail + oldTailSize + ~TAILBLOCK_MASK) & TAILBLOCK_MASK; |
229 | - final int newNext = (tail + newTailSize + ~TAILBLOCK_MASK) & TAILBLOCK_MASK; |
230 | - if (newNext < oldNext) { |
231 | - // Free the remainder of the old tail block |
232 | - deallocTail(newNext, oldNext - newNext); |
233 | - } else { |
234 | - Debug.$assert0.t(newNext == oldNext); |
235 | - } |
236 | - // Rewrite the tail block header |
237 | - putInt(tail, encodeTailBlock(newTailSize, klength)); |
238 | - if (Debug.ENABLED) { |
239 | - MVV.verify(_bytes, offset, newSize); |
240 | - } |
241 | - } |
242 | - valueByte = newSize > 0 ? _bytes[offset] & 0xFF : -1; |
243 | - incCountIfMvv(_bytes, offset, newSize); |
244 | - } |
245 | - |
246 | - if (pruneAntiValue(valueByte, p, tree)) { |
247 | - changed = true; |
248 | - p -= KEYBLOCK_LENGTH; |
249 | - } |
250 | - } |
251 | - } |
252 | + final int flags = pruneMvvValuesHelper(tree); |
253 | + changed = (flags & PRUNE_MVV_HELPER_CHANGED) != 0; |
254 | + hasLongMvvRecords = (flags & PRUNE_MVV_HELPER_HAS_LONG) != 0; |
255 | + |
256 | if (changed) { |
257 | setDirtyAtTimestamp(timestamp); |
258 | } |
259 | |
260 | - deallocatePrunedVersions(_persistit, _vol, prunedVersions); |
261 | - prunedVersions.clear(); |
262 | + final List<PrunedVersion> prunedVersions = new ArrayList<PrunedVersion>(); |
263 | |
264 | if (pruneLongMVVs && hasLongMvvRecords) { |
265 | final List<PersistitException> deferredExceptions = new ArrayList<PersistitException>(); |
266 | @@ -3645,6 +3622,63 @@ |
267 | return changed; |
268 | } |
269 | |
270 | + private int pruneMvvValuesHelper(final Tree tree) throws PersistitException { |
271 | + boolean changed = false; |
272 | + boolean hasLongMvvRecords = false; |
273 | + final List<PrunedVersion> prunedVersions = new ArrayList<PrunedVersion>(); |
274 | + |
275 | + for (int p = KEY_BLOCK_START; p < _keyBlockEnd; p += KEYBLOCK_LENGTH) { |
276 | + final int kbData = getInt(p); |
277 | + final int tail = decodeKeyBlockTail(kbData); |
278 | + final int tbData = getInt(tail); |
279 | + final int klength = decodeTailBlockKLength(tbData); |
280 | + final int oldTailSize = decodeTailBlockSize(tbData); |
281 | + final int offset = tail + _tailHeaderSize + klength; |
282 | + final int oldSize = oldTailSize - klength - _tailHeaderSize; |
283 | + |
284 | + if (oldSize > 0) { |
285 | + int valueByte = _bytes[offset] & 0xFF; |
286 | + if (isLongMVV(_bytes, offset, oldSize)) { |
287 | + /* |
288 | + * Can't prune in this pass because of long record timestamp |
289 | + * management. Simply remember that there are long records |
290 | + * to prune, then prune them in a copy of the buffer. |
291 | + */ |
292 | + hasLongMvvRecords = true; |
293 | + } |
294 | + if (valueByte == MVV.TYPE_MVV) { |
295 | + final int newSize = MVV.prune(_bytes, offset, oldSize, _persistit.getTransactionIndex(), true, |
296 | + prunedVersions); |
297 | + if (newSize != oldSize) { |
298 | + changed = true; |
299 | + final int newTailSize = klength + newSize + _tailHeaderSize; |
300 | + final int oldNext = (tail + oldTailSize + ~TAILBLOCK_MASK) & TAILBLOCK_MASK; |
301 | + final int newNext = (tail + newTailSize + ~TAILBLOCK_MASK) & TAILBLOCK_MASK; |
302 | + if (newNext < oldNext) { |
303 | + // Free the remainder of the old tail block |
304 | + deallocTail(newNext, oldNext - newNext); |
305 | + } else { |
306 | + Debug.$assert0.t(newNext == oldNext); |
307 | + } |
308 | + // Rewrite the tail block header |
309 | + putInt(tail, encodeTailBlock(newTailSize, klength)); |
310 | + Debug.$assert0.t(MVV.verify(_bytes, offset, newSize)); |
311 | + } |
312 | + valueByte = newSize > 0 ? _bytes[offset] & 0xFF : -1; |
313 | + incCountIfMvv(_bytes, offset, newSize); |
314 | + } |
315 | + |
316 | + if (pruneAntiValue(valueByte, p, tree)) { |
317 | + changed = true; |
318 | + p -= KEYBLOCK_LENGTH; |
319 | + } |
320 | + } |
321 | + } |
322 | + deallocatePrunedVersions(_persistit, _vol, prunedVersions); |
323 | + prunedVersions.clear(); |
324 | + return (changed ? PRUNE_MVV_HELPER_CHANGED : 0) | (hasLongMvvRecords ? PRUNE_MVV_HELPER_HAS_LONG : 0); |
325 | + } |
326 | + |
327 | /** |
328 | * This method attempts to prune each LongRecord MVVs found in the page. |
329 | * This method is performed on a _copy_ of the original Buffer being pruned; |
330 | @@ -3745,7 +3779,7 @@ |
331 | final byte[] rawBytes = value.getEncodedBytes(); |
332 | final int oldLongSize = value.getEncodedSize(); |
333 | // TODO - perhaps remove. Done as a precaution for now. |
334 | - MVV.verify(rawBytes, 0, oldLongSize); |
335 | + Debug.$assert0.t(MVV.verify(rawBytes, 0, oldLongSize)); |
336 | final List<PrunedVersion> provisionalPrunedVersions = new ArrayList<PrunedVersion>(); |
337 | final int newLongSize = MVV.prune(rawBytes, 0, oldLongSize, _persistit.getTransactionIndex(), true, |
338 | provisionalPrunedVersions); |
339 | |
340 | === modified file 'src/main/java/com/persistit/BufferPool.java' |
341 | --- src/main/java/com/persistit/BufferPool.java 2012-08-24 14:04:53 +0000 |
342 | +++ src/main/java/com/persistit/BufferPool.java 2012-08-24 14:04:53 +0000 |
343 | @@ -15,16 +15,14 @@ |
344 | |
345 | package com.persistit; |
346 | |
347 | -import java.io.BufferedReader; |
348 | -import java.io.BufferedWriter; |
349 | import java.io.DataOutputStream; |
350 | -import java.io.File; |
351 | -import java.io.FileReader; |
352 | -import java.io.FileWriter; |
353 | import java.io.IOException; |
354 | import java.nio.ByteBuffer; |
355 | +import java.util.ArrayList; |
356 | import java.util.Arrays; |
357 | +import java.util.Collections; |
358 | import java.util.HashSet; |
359 | +import java.util.List; |
360 | import java.util.Set; |
361 | import java.util.concurrent.atomic.AtomicBoolean; |
362 | import java.util.concurrent.atomic.AtomicInteger; |
363 | @@ -32,6 +30,7 @@ |
364 | import java.util.concurrent.atomic.AtomicLongArray; |
365 | import java.util.concurrent.locks.ReentrantLock; |
366 | |
367 | +import com.persistit.JournalManager.PageNode; |
368 | import com.persistit.exception.InUseException; |
369 | import com.persistit.exception.InvalidPageAddressException; |
370 | import com.persistit.exception.InvalidPageStructureException; |
371 | @@ -92,6 +91,17 @@ |
372 | */ |
373 | private final static int WRITE_AGE_THRESHOLD_RATIO = 4; |
374 | |
375 | + private final static String INVENTORY_TREE_NAME = "_buffers"; |
376 | + /** |
377 | + * Maximum number of buffer inventory versions to retain |
378 | + */ |
379 | + private final static int INVENTORY_VERSIONS = 3; |
380 | + |
381 | + /** |
382 | + * Preload log multiple |
383 | + */ |
384 | + private final static int INVENTORY_PRELOAD_LOG_MESSAGE_MULTIPLE = 10000; |
385 | + |
386 | /** |
387 | * The Persistit instance that references this BufferPool. |
388 | */ |
389 | @@ -203,23 +213,11 @@ |
390 | private volatile int _pageWriterTrancheSize = PAGE_WRITER_TRANCHE_SIZE; |
391 | |
392 | /** |
393 | - * Polling interval for PageCacher |
394 | - */ |
395 | - private volatile long _cacherPollInterval; |
396 | - |
397 | - /** |
398 | * The PAGE_WRITER IOTaskRunnable |
399 | */ |
400 | private PageWriter _writer; |
401 | |
402 | /** |
403 | - * The PAGE_CACHER IOTaskRunnable |
404 | - */ |
405 | - private PageCacher _cacher; |
406 | - |
407 | - private String _defaultLogPath; |
408 | - |
409 | - /** |
410 | * Construct a BufferPool with the specified count of <code>Buffer</code>s |
411 | * of the specified size. |
412 | * |
413 | @@ -289,37 +287,6 @@ |
414 | throw e; |
415 | } |
416 | _writer = new PageWriter(); |
417 | - _cacher = new PageCacher(); |
418 | - } |
419 | - |
420 | - void warmupBufferPool(final String pathName, final String fname) throws PersistitException { |
421 | - final File file = new File(pathName, fname + ".log"); |
422 | - _defaultLogPath = file.getAbsolutePath(); |
423 | - |
424 | - try { |
425 | - if (!file.exists()) { |
426 | - file.createNewFile(); |
427 | - } |
428 | - |
429 | - final BufferedReader reader = new BufferedReader(new FileReader(file)); |
430 | - String currLine; |
431 | - while ((currLine = reader.readLine()) != null) { |
432 | - final String[] info = currLine.split(" "); |
433 | - if (info.length == 2) { |
434 | - final Volume vol = _persistit.getVolume(info[1]); |
435 | - if (vol != null) { |
436 | - final long page = Long.parseLong(info[0]); |
437 | - final Buffer buff = get(vol, page, false, true); |
438 | - buff.release(); |
439 | - } |
440 | - } |
441 | - } |
442 | - reader.close(); |
443 | - _cacherPollInterval = _persistit.getConfiguration().getBufferInventoryPollingInterval(); |
444 | - _cacher.start(); |
445 | - } catch (final IOException e) { |
446 | - throw new PersistitException(e); |
447 | - } |
448 | } |
449 | |
450 | void startThreads() throws PersistitException { |
451 | @@ -329,9 +296,7 @@ |
452 | void close() { |
453 | _closed.set(true); |
454 | _persistit.waitForIOTaskStop(_writer); |
455 | - _persistit.waitForIOTaskStop(_cacher); |
456 | _writer = null; |
457 | - _cacher = null; |
458 | } |
459 | |
460 | /** |
461 | @@ -340,7 +305,6 @@ |
462 | */ |
463 | void crash() { |
464 | IOTaskRunnable.crash(_writer); |
465 | - IOTaskRunnable.crash(_cacher); |
466 | } |
467 | |
468 | void flush(final long timestamp) throws PersistitInterruptedException { |
469 | @@ -433,35 +397,6 @@ |
470 | } |
471 | } |
472 | |
473 | - private void populateWarmupFile() throws PersistitException { |
474 | - final File file = new File(_defaultLogPath); |
475 | - |
476 | - try { |
477 | - final BufferedWriter writer = new BufferedWriter(new FileWriter(file)); |
478 | - for (int i = 0; i < _buffers.length; ++i) { |
479 | - final Buffer b = _buffers[i]; |
480 | - if (b != null && b.isValid() && !b.isDirty()) { |
481 | - final long page = b.getPageAddress(); |
482 | - final Volume volume = b.getVolume(); |
483 | - final long page2 = b.getPageAddress(); |
484 | - final Volume volume2 = b.getVolume(); |
485 | - |
486 | - // Check if buffer has changed while reading |
487 | - if (page == page2 && volume == volume2 && volume != null) { |
488 | - final String addr = Long.toString(page); |
489 | - final String vol = volume.getName(); |
490 | - writer.append(addr + " " + vol); |
491 | - writer.newLine(); |
492 | - writer.flush(); |
493 | - } |
494 | - } |
495 | - } |
496 | - writer.close(); |
497 | - } catch (final IOException e) { |
498 | - throw new PersistitException(e); |
499 | - } |
500 | - } |
501 | - |
502 | private boolean selected(final Buffer buffer, final int includeMask, final int excludeMask) { |
503 | return ((includeMask == 0) || (buffer.getStatus() & includeMask) != 0) |
504 | && (buffer.getStatus() & excludeMask) == 0; |
505 | @@ -1403,35 +1338,6 @@ |
506 | } |
507 | } |
508 | |
509 | - /** |
510 | - * Implementation of PAGE_CACHER thread |
511 | - */ |
512 | - class PageCacher extends IOTaskRunnable { |
513 | - |
514 | - PageCacher() { |
515 | - super(BufferPool.this._persistit); |
516 | - } |
517 | - |
518 | - void start() { |
519 | - start("PAGE_CACHER:" + _bufferSize, _cacherPollInterval); |
520 | - } |
521 | - |
522 | - @Override |
523 | - public void runTask() throws Exception { |
524 | - populateWarmupFile(); |
525 | - } |
526 | - |
527 | - @Override |
528 | - protected boolean shouldStop() { |
529 | - return _closed.get() && !isFlushing(); |
530 | - } |
531 | - |
532 | - @Override |
533 | - protected long pollInterval() { |
534 | - return isFlushing() ? 0 : _cacherPollInterval; |
535 | - } |
536 | - } |
537 | - |
538 | @Override |
539 | public String toString() { |
540 | return "BufferPool[" + _bufferCount + "@" + _bufferSize + (_closed.get() ? ":closed" : "") + "]"; |
541 | @@ -1486,4 +1392,133 @@ |
542 | } |
543 | stream.flush(); |
544 | } |
545 | + |
546 | + void recordBufferInventory(final long timestamp) throws PersistitException { |
547 | + final Exchange exchange = getBufferInventoryExchange(); |
548 | + /* |
549 | + * Advisory only - transaction integrity not needed |
550 | + */ |
551 | + exchange.ignoreTransactions(); |
552 | + try { |
553 | + int total = 0; |
554 | + exchange.clear().append(_bufferSize).append(timestamp).append(Key.BEFORE); |
555 | + final Value value = exchange.getValue(); |
556 | + final int clockValueBefore = _clock.get(); |
557 | + for (int index = 0; index < _buffers.length; index++) { |
558 | + final Buffer buffer = _buffers[index]; |
559 | + long page1 = -1, page2 = -1; |
560 | + Volume volume1 = null, volume2 = null; |
561 | + if (buffer != null && buffer.isValid()) { |
562 | + while (true) { |
563 | + page1 = buffer.getPageAddress(); |
564 | + volume1 = buffer.getVolume(); |
565 | + page2 = buffer.getPageAddress(); |
566 | + volume2 = buffer.getVolume(); |
567 | + if (page1 == page2 && volume1 == volume2) { |
568 | + break; |
569 | + } |
570 | + Util.spinSleep(); |
571 | + } |
572 | + if (volume1 != null && !volume1.isTemporary()) { |
573 | + value.clear().setStreamMode(true); |
574 | + value.put(volume1.getHandle()); |
575 | + value.put(page1); |
576 | + exchange.to(index).store(); |
577 | + total++; |
578 | + } |
579 | + } |
580 | + } |
581 | + final int clockValueAfter = _clock.get(); |
582 | + exchange.cut(); |
583 | + value.clear().setStreamMode(true); |
584 | + value.put(_bufferCount); |
585 | + value.put(total); |
586 | + value.put(clockValueBefore); |
587 | + value.put(clockValueAfter); |
588 | + value.put(System.currentTimeMillis()); |
589 | + exchange.store(); |
590 | + int count = 0; |
591 | + while (exchange.previous()) { |
592 | + if (++count > INVENTORY_VERSIONS) { |
593 | + exchange.remove(Key.GTEQ); |
594 | + } |
595 | + } |
596 | + } catch (final PersistitException e) { |
597 | + _persistit.getLogBase().bufferInventoryException.log(e); |
598 | + } |
599 | + } |
600 | + |
601 | + void preloadBufferInventory() { |
602 | + int count = 0; |
603 | + int total = 0; |
604 | + try { |
605 | + final JournalManager jman = _persistit.getJournalManager(); |
606 | + final Exchange exchange = getBufferInventoryExchange(); |
607 | + final Value value = exchange.getValue(); |
608 | + final List<PageNode> pageNodes = new ArrayList<PageNode>(); |
609 | + boolean foundInventory = false; |
610 | + exchange.clear().append(_bufferSize).append(Key.AFTER); |
611 | + while (exchange.previous()) { |
612 | + if (exchange.getValue().isDefined()) { |
613 | + foundInventory = true; |
614 | + break; |
615 | + } |
616 | + } |
617 | + if (!foundInventory) { |
618 | + return; |
619 | + } |
620 | + value.setStreamMode(true); |
621 | + /* int bufferCount = */value.getInt(); |
622 | + total = value.getInt(); |
623 | + /* int clockValueBefore = */value.getInt(); |
624 | + /* int clockValueAfter = */value.getInt(); |
625 | + final long systemTime = value.getLong(); |
626 | + |
627 | + _persistit.getLogBase().bufferInventoryLoad.log(systemTime); |
628 | + |
629 | + exchange.append(Key.BEFORE); |
630 | + |
631 | + while (exchange.next()) { |
632 | + value.setStreamMode(true); |
633 | + final int volumeHandle = value.getInt(); |
634 | + final long pageAddress = value.getLong(); |
635 | + final PageNode pn = new PageNode(volumeHandle, pageAddress); |
636 | + pageNodes.add(pn); |
637 | + } |
638 | + |
639 | + Collections.sort(pageNodes, PageNode.READ_COMPARATOR); |
640 | + for (final PageNode pn : pageNodes) { |
641 | + final Volume vol = jman.volumeForHandle(pn.getVolumeHandle()); |
642 | + if (vol == null) { |
643 | + continue; |
644 | + } |
645 | + try { |
646 | + final Buffer buff = get(vol, pn.getPageAddress(), false, true); |
647 | + buff.release(); |
648 | + count++; |
649 | + if ((count % INVENTORY_PRELOAD_LOG_MESSAGE_MULTIPLE) == 0) { |
650 | + _persistit.getLogBase().bufferInventoryProgress.log(count, total); |
651 | + } |
652 | + if (count >= _bufferCount) { |
653 | + // |
654 | + // If the buffer pool is now smaller, no need to load |
655 | + // more pages |
656 | + // |
657 | + break; |
658 | + } |
659 | + } catch (final PersistitException e) { |
660 | + // ignore it |
661 | + } |
662 | + } |
663 | + } catch (final PersistitException e) { |
664 | + _persistit.getLogBase().bufferInventoryException.log(e); |
665 | + } finally { |
666 | + _persistit.getLogBase().bufferInventoryProgress.log(count, total); |
667 | + } |
668 | + } |
669 | + |
670 | + private Exchange getBufferInventoryExchange() throws PersistitException { |
671 | + final Volume sysvol = _persistit.getSystemVolume(); |
672 | + return _persistit.getExchange(sysvol, INVENTORY_TREE_NAME, true); |
673 | + } |
674 | } |
675 | |
676 | === modified file 'src/main/java/com/persistit/CLI.java' |
677 | --- src/main/java/com/persistit/CLI.java 2012-08-24 14:04:53 +0000 |
678 | +++ src/main/java/com/persistit/CLI.java 2012-08-24 14:04:53 +0000 |
679 | @@ -747,7 +747,7 @@ |
680 | } |
681 | } |
682 | if (sysvol != null) { |
683 | - properties.put(Persistit.SYSTEM_VOLUME_PROPERTY, sysvol.getName()); |
684 | + persistit.getConfiguration().setSysVolume(sysvol.getName()); |
685 | } |
686 | |
687 | _persistit = persistit; |
688 | |
689 | === modified file 'src/main/java/com/persistit/CheckpointManager.java' |
690 | --- src/main/java/com/persistit/CheckpointManager.java 2012-08-24 14:04:53 +0000 |
691 | +++ src/main/java/com/persistit/CheckpointManager.java 2012-08-24 14:04:53 +0000 |
692 | @@ -201,6 +201,7 @@ |
693 | void pollCreateCheckpoint() throws PersistitException { |
694 | final long now = System.nanoTime(); |
695 | if (_lastCheckpointNanos + _checkpointIntervalNanos < now) { |
696 | + _persistit.recordBufferPoolInventory(); |
697 | createCheckpoint(); |
698 | } |
699 | } |
700 | @@ -252,12 +253,12 @@ |
701 | _currentCheckpoint = new Checkpoint(txn.getStartTimestamp(), System.currentTimeMillis()); |
702 | _outstandingCheckpoints.add(_currentCheckpoint); |
703 | _persistit.getLogBase().checkpointProposed.log(_currentCheckpoint); |
704 | - return _currentCheckpoint; |
705 | } catch (final InterruptedException ie) { |
706 | throw new PersistitInterruptedException(ie); |
707 | } finally { |
708 | txn.end(); |
709 | } |
710 | + return _currentCheckpoint; |
711 | } finally { |
712 | _persistit.setSessionId(saveSessionId); |
713 | } |
714 | |
715 | === modified file 'src/main/java/com/persistit/CleanupManager.java' |
716 | --- src/main/java/com/persistit/CleanupManager.java 2012-08-24 14:04:53 +0000 |
717 | +++ src/main/java/com/persistit/CleanupManager.java 2012-08-24 14:04:53 +0000 |
718 | @@ -87,6 +87,7 @@ |
719 | _accepted.incrementAndGet(); |
720 | } else { |
721 | _refused.incrementAndGet(); |
722 | + kick(); |
723 | } |
724 | return accepted; |
725 | } |
726 | @@ -127,6 +128,15 @@ |
727 | } |
728 | |
729 | @Override |
730 | + public long getPollInterval() { |
731 | + if (_cleanupActionQueue.size() < DEFAULT_QUEUE_SIZE / 2) { |
732 | + return super.getPollInterval(); |
733 | + } else { |
734 | + return 0; |
735 | + } |
736 | + } |
737 | + |
738 | + @Override |
739 | public void poll() throws Exception { |
740 | _persistit.getIOMeter().poll(); |
741 | _persistit.cleanup(); |
742 | |
743 | === modified file 'src/main/java/com/persistit/Configuration.java' |
744 | --- src/main/java/com/persistit/Configuration.java 2012-08-24 14:04:53 +0000 |
745 | +++ src/main/java/com/persistit/Configuration.java 2012-08-24 14:04:53 +0000 |
746 | @@ -263,7 +263,7 @@ |
747 | /** |
748 | * Property name for the "append only" property. |
749 | */ |
750 | - public final static String APPEND_ONLY_PROPERTY = "appendonly"; |
751 | + public final static String APPEND_ONLY_PROPERTY_NAME = "appendonly"; |
752 | |
753 | /** |
754 | * Property name for the "ignore missing volumes" property. |
755 | @@ -276,12 +276,15 @@ |
756 | public final static String SPLIT_POLICY_PROPERTY_NAME = "splitpolicy"; |
757 | |
758 | /** |
759 | - * Property name to specify the"buffer inventory" property name. |
760 | + * Property name to specify whether buffer preloading is enabled. |
761 | + */ |
762 | + public final static String BUFFER_PRELOAD_PROPERTY_NAME = "bufferpreload"; |
763 | + |
764 | + /** |
765 | + * Property name to specify whether buffer inventory is enabled. |
766 | */ |
767 | public final static String BUFFER_INVENTORY_PROPERTY_NAME = "bufferinventory"; |
768 | |
769 | - public final static String BUFFER_POLLING_INTERVAL_PROPERTY = "bufferpollinginterval"; |
770 | - |
771 | /** |
772 | * Property name to specify the default {@link JoinPolicy}. |
773 | */ |
774 | @@ -634,9 +637,8 @@ |
775 | private int rmiServerPort; |
776 | private boolean jmx = true; |
777 | private boolean appendOnly; |
778 | - private String bufferInventoryPathName; |
779 | - private long bufferInventoryPollInterval = 3000000; // default five minute |
780 | - // polling |
781 | + private boolean bufferInventoryEnabled; |
782 | + private boolean bufferPreloadEnabled; |
783 | private boolean ignoreMissingVolumes; |
784 | private String tmpVolDir; |
785 | private int tmpVolPageSize; |
786 | @@ -715,9 +717,7 @@ |
787 | } |
788 | |
789 | void loadProperties() throws InvalidVolumeSpecificationException { |
790 | - setBufferInventoryPathName(getProperty(BUFFER_INVENTORY_PROPERTY_NAME)); |
791 | - setBufferInventoryPollingInterval(getLongProperty(BUFFER_POLLING_INTERVAL_PROPERTY, bufferInventoryPollInterval)); |
792 | - setAppendOnly(getBooleanProperty(APPEND_ONLY_PROPERTY, false)); |
793 | + setAppendOnly(getBooleanProperty(APPEND_ONLY_PROPERTY_NAME, false)); |
794 | setCommitPolicy(getProperty(COMMIT_POLICY_PROPERTY_NAME)); |
795 | setConstructorOverride(getBooleanProperty(CONSTRUCTOR_OVERRIDE_PROPERTY_NAME, false)); |
796 | setIgnoreMissingVolumes(getBooleanProperty(IGNORE_MISSING_VOLUMES_PROPERTY, false)); |
797 | @@ -737,6 +737,8 @@ |
798 | setShowGUI(getBooleanProperty(SHOW_GUI_PROPERTY_NAME, false)); |
799 | setSplitPolicy(getProperty(SPLIT_POLICY_PROPERTY_NAME)); |
800 | setSysVolume(getProperty(SYSTEM_VOLUME_PROPERTY_NAME, DEFAULT_SYSTEM_VOLUME_NAME)); |
801 | + setBufferInventoryEnabled(getBooleanProperty(BUFFER_INVENTORY_PROPERTY_NAME, false)); |
802 | + setBufferPreloadEnabled(getBooleanProperty(BUFFER_PRELOAD_PROPERTY_NAME, false)); |
803 | |
804 | loadPropertiesBufferSpecifications(); |
805 | loadPropertiesVolumeSpecifications(); |
806 | @@ -1548,8 +1550,7 @@ |
807 | /** |
808 | * <p> |
809 | * Set a pattern that identifies classes to be serialized using standard |
810 | - * Java serialization rather than Persistit's default serialization. TODO |
811 | - * Link to Serialization section of user_guide.html. |
812 | + * Java serialization rather than Persistit's default serialization. |
813 | * </p> |
814 | * <p> |
815 | * Default value is <code>null</code><br /> |
816 | @@ -1559,6 +1560,8 @@ |
817 | * @param serialOverride |
818 | * the serial override pattern to set |
819 | * @see DefaultCoderManager |
820 | + * @see <a |
821 | + * href="http://www.akiban.com/ak-docs/admin/persistit/Serialization.html">Serialization</a> |
822 | */ |
823 | public void setSerialOverride(final String serialOverride) { |
824 | this.serialOverride = serialOverride; |
825 | @@ -1579,8 +1582,7 @@ |
826 | * a public no-argument constructor. If so, then that constructor is used |
827 | * when deserializing in the {@link DefaultObjectCoder}; if not then |
828 | * Persistit uses private methods within the JDK to emulate standard Java |
829 | - * serialization logic. TODO Link to Serialization section of |
830 | - * user_guide.html. |
831 | + * serialization logic. |
832 | * </p> |
833 | * <p> |
834 | * Default value is <code>false</code><br /> |
835 | @@ -1589,6 +1591,8 @@ |
836 | * |
837 | * @param constructorOverride |
838 | * the constructorOverride to set |
839 | + * @see <a |
840 | + * href="http://www.akiban.com/ak-docs/admin/persistit/Serialization.html">Serialization</a> |
841 | */ |
842 | public void setConstructorOverride(final boolean constructorOverride) { |
843 | this.constructorOverride = constructorOverride; |
844 | @@ -1791,7 +1795,7 @@ |
845 | /** |
846 | * Return the value defined by {@link #setAppendOnly} |
847 | * |
848 | - * @return the whether to start Persistit in append-only mode |
849 | + * @return <true>true</code> if append-only mode is enabled at startup |
850 | */ |
851 | public boolean isAppendOnly() { |
852 | return appendOnly; |
853 | @@ -1807,77 +1811,73 @@ |
854 | * </p> |
855 | * <p> |
856 | * Default value is <code>false</code><br /> |
857 | - * Property name is {@value #APPEND_ONLY_PROPERTY} |
858 | + * Property name is {@value #APPEND_ONLY_PROPERTY_NAME} |
859 | * </p> |
860 | * |
861 | * @param appendOnly |
862 | - * <code>true</code> to start Persistit in append-only only |
863 | + * <code>true</code> to start Persistit in append-only mode |
864 | */ |
865 | public void setAppendOnly(final boolean appendOnly) { |
866 | this.appendOnly = appendOnly; |
867 | } |
868 | |
869 | /** |
870 | - * Return the path name defined by {@link #getBufferInventoryPathName} |
871 | - * |
872 | - * @return the path where file to warm-up Persistit with sample buffer data |
873 | - * is stored |
874 | - */ |
875 | - public String getBufferInventoryPathName() { |
876 | - return bufferInventoryPathName; |
877 | - } |
878 | - |
879 | - /** |
880 | - * <p> |
881 | - * Control where Persistit stores its buffer inventory. In this mode |
882 | - * Persistit restarts with information from the last run. This method |
883 | - * initializes the warm-up file at the specified location, if none is |
884 | - * specified the buffer pool is not warmed up on start-up. |
885 | - * </p> |
886 | - * <p> |
887 | - * Default value is <code>null</code><br /> |
888 | - * Property name is {@value #BUFFER_INVENTORY_PROPERTY_NAME} |
889 | - * </p> |
890 | - * |
891 | - * @param pathName |
892 | - * the name of the path to the warm-up file |
893 | - */ |
894 | - public void setBufferInventoryPathName(final String pathName) { |
895 | - bufferInventoryPathName = pathName; |
896 | - |
897 | - } |
898 | - |
899 | - /** |
900 | - * Return polling interval defined by |
901 | - * {@link #getBufferInventoryPollingInterval} |
902 | - * |
903 | - * @return the number of seconds wait between warm-up polls |
904 | - */ |
905 | - public long getBufferInventoryPollingInterval() { |
906 | - return bufferInventoryPollInterval; |
907 | - } |
908 | - |
909 | - /** |
910 | - * <p> |
911 | - * Control the number of seconds between each poll for the cache warm-up |
912 | - * option in Persistit. |
913 | - * </p> |
914 | - * <p> |
915 | - * Default value is <code>3000</code><br /> |
916 | - * Property name is {@value #BUFFER_POLLING_INTERVAL_PROPERTY} |
917 | - * </p> |
918 | - * |
919 | - * @param seconds |
920 | - * the number of seconds between polls |
921 | - */ |
922 | - public void setBufferInventoryPollingInterval(final long seconds) { |
923 | - bufferInventoryPollInterval = Util.rangeCheck(seconds, 60L, Long.MAX_VALUE) * 1000L; |
924 | + * Return the value defined by {@link #setBufferInventoryEnabled} |
925 | + * |
926 | + * @return <code>true</code> if periodic buffer pool inventory recording is |
927 | + * enabled |
928 | + */ |
929 | + public boolean isBufferInventoryEnabled() { |
930 | + return bufferInventoryEnabled; |
931 | + } |
932 | + |
933 | + /** |
934 | + * <p> |
935 | + * Control whether Persistit periodically records an inventory of its buffer |
936 | + * pools to enable buffer pool preloading on a subsequent startup. |
937 | + * </p> |
938 | + * <p> |
939 | + * Default value is <code>false</code><br /> |
940 | + * Property name is {@value #BUFFER_INVENTORY_PROPERTY_NAME} |
941 | + * |
942 | + * @param bufferInventoryEnabled |
943 | + * <code>true</code> to enable periodic buffer inventory behavior |
944 | + */ |
945 | + public void setBufferInventoryEnabled(final boolean bufferInventoryEnabled) { |
946 | + this.bufferInventoryEnabled = bufferInventoryEnabled; |
947 | + } |
948 | + |
949 | + /** |
950 | + * Return the value defined by {@link #setBufferPreloadEnabled} |
951 | + * |
952 | + * @return <code>true</code> if the option to preload buffer pools is |
953 | + * enabled |
954 | + */ |
955 | + public boolean isBufferPreloadEnabled() { |
956 | + return bufferPreloadEnabled; |
957 | + } |
958 | + |
959 | + /** |
960 | + * <p> |
961 | + * Control whether Persistit attempts to preload (warm up) the buffer pools |
962 | + * by preloading pages recorded in a previously generated inventory. |
963 | + * </p> |
964 | + * <p> |
965 | + * Default value is <code>false</code><br /> |
966 | + * Property name is {@value #BUFFER_INVENTORY_PROPERTY_NAME} |
967 | + * |
968 | + * @param bufferPreloadEnabled |
969 | + * <code>true</code> to enable buffer pool preloading during |
970 | + * startup |
971 | + */ |
972 | + public void setBufferPreloadEnabled(final boolean bufferPreloadEnabled) { |
973 | + this.bufferPreloadEnabled = bufferPreloadEnabled; |
974 | } |
975 | |
976 | /** |
977 | * Return the value defined by {@link #setIgnoreMissingVolumes(boolean)} |
978 | * |
979 | - * @return the whether to start Persistit in ignore-missing-volumes mode |
980 | + * @return <code>true</code>to enable ignore-missing-volumes mode |
981 | */ |
982 | public boolean isIgnoreMissingVolumes() { |
983 | return ignoreMissingVolumes; |
984 | |
985 | === modified file 'src/main/java/com/persistit/Exchange.java' |
986 | --- src/main/java/com/persistit/Exchange.java 2012-08-24 14:04:53 +0000 |
987 | +++ src/main/java/com/persistit/Exchange.java 2012-08-24 14:04:53 +0000 |
988 | @@ -1529,6 +1529,9 @@ |
989 | storedLength &= MVV.STORE_LENGTH_MASK; |
990 | spareValue.setEncodedSize(storedLength); |
991 | |
992 | + // TODO remove after debugging |
993 | + MVV.verify(_persistit.getTransactionIndex(), spareBytes, 0, storedLength); |
994 | + |
995 | if (spareValue.getEncodedSize() > maxSimpleValueSize) { |
996 | newLongRecordPointerMVV = getLongRecordHelper().storeLongRecord(spareValue, |
997 | _transaction.isActive()); |
998 | @@ -3698,6 +3701,8 @@ |
999 | buffer.nextKey(_spareKey2, buffer.toKeyBlock(0)); |
1000 | _value.setPointerValue(page); |
1001 | _value.setPointerPageType(buffer.getPageType()); |
1002 | + buffer.release(); |
1003 | + buffer = null; |
1004 | storeInternal(_spareKey2, _value, level + 1, Exchange.StoreOptions.NONE); |
1005 | return true; |
1006 | } finally { |
1007 | |
1008 | === modified file 'src/main/java/com/persistit/IOMeter.java' |
1009 | --- src/main/java/com/persistit/IOMeter.java 2012-08-24 14:04:53 +0000 |
1010 | +++ src/main/java/com/persistit/IOMeter.java 2012-08-24 14:04:53 +0000 |
1011 | @@ -61,15 +61,16 @@ |
1012 | |
1013 | private final static int READ_PAGE_FROM_VOLUME = 1; |
1014 | private final static int READ_PAGE_FROM_JOURNAL = 2; |
1015 | - private final static int COPY_PAGE_TO_VOLUME = 3; |
1016 | - private final static int WRITE_PAGE_TO_JOURNAL = 4; |
1017 | - private final static int WRITE_TX_TO_JOURNAL = 5; |
1018 | - private final static int WRITE_OTHER_TO_JOURNAL = 6; |
1019 | - private final static int EVICT_PAGE_FROM_POOL = 7; |
1020 | - private final static int FLUSH_JOURNAL = 8; |
1021 | - private final static int GET_PAGE = 9; |
1022 | + private final static int COPY_PAGE_FROM_JOURNAL = 3; |
1023 | + private final static int COPY_PAGE_TO_VOLUME = 4; |
1024 | + private final static int WRITE_PAGE_TO_JOURNAL = 5; |
1025 | + private final static int WRITE_TX_TO_JOURNAL = 6; |
1026 | + private final static int WRITE_OTHER_TO_JOURNAL = 7; |
1027 | + private final static int EVICT_PAGE_FROM_POOL = 8; |
1028 | + private final static int FLUSH_JOURNAL = 9; |
1029 | + private final static int GET_PAGE = 10; |
1030 | |
1031 | - private final static int ITEM_COUNT = 10; |
1032 | + private final static int ITEM_COUNT = 11; |
1033 | |
1034 | private long _quiescentIOthreshold = DEFAULT_QUIESCENT_IO_THRESHOLD; |
1035 | |
1036 | @@ -266,7 +267,7 @@ |
1037 | |
1038 | if (analyzePages |
1039 | && (op == WRITE_PAGE_TO_JOURNAL || op == READ_PAGE_FROM_JOURNAL || op == READ_PAGE_FROM_VOLUME |
1040 | - || op == COPY_PAGE_TO_VOLUME || op == EVICT_PAGE_FROM_POOL)) { |
1041 | + || op == COPY_PAGE_FROM_JOURNAL || op == COPY_PAGE_TO_VOLUME || op == EVICT_PAGE_FROM_POOL)) { |
1042 | final long handle = (volumeHandle << 48) + pageAddress; |
1043 | List<Event> list = events.get(handle); |
1044 | if (list == null) { |
1045 | @@ -291,6 +292,12 @@ |
1046 | } |
1047 | } |
1048 | |
1049 | + public void chargeCopyPageFromJournal(final Volume volume, final long pageAddress, final int size, |
1050 | + final long journalAddress, final int urgency) { |
1051 | + charge(size, COPY_PAGE_FROM_JOURNAL); |
1052 | + log(COPY_PAGE_FROM_JOURNAL, volume, pageAddress, size, journalAddress, 0); |
1053 | + } |
1054 | + |
1055 | public void chargeCopyPageToVolume(final Volume volume, final long pageAddress, final int size, |
1056 | final long journalAddress, final int urgency) { |
1057 | charge(size, COPY_PAGE_TO_VOLUME); |
1058 | |
1059 | === modified file 'src/main/java/com/persistit/JournalManager.java' |
1060 | --- src/main/java/com/persistit/JournalManager.java 2012-08-24 14:04:53 +0000 |
1061 | +++ src/main/java/com/persistit/JournalManager.java 2012-08-24 14:04:53 +0000 |
1062 | @@ -77,6 +77,7 @@ |
1063 | final static int URGENT = 10; |
1064 | final static int ALMOST_URGENT = 8; |
1065 | final static int HALF_URGENT = 5; |
1066 | + final static int URGENT_COMMIT_DELAY = 50; // delay in ms |
1067 | private final static long NS_PER_MS = 1000000L; |
1068 | private final static int IO_MEASUREMENT_CYCLES = 8; |
1069 | |
1070 | @@ -140,13 +141,13 @@ |
1071 | * requires more space than is available in the current journal file, it |
1072 | * will advance to the start of the next journal file. |
1073 | */ |
1074 | - private long _currentAddress; |
1075 | + private volatile long _currentAddress; |
1076 | |
1077 | /** |
1078 | * Smallest journal address at which a record still needed is located. |
1079 | * Initially zero, increases as journal files are consumed and deleted. |
1080 | */ |
1081 | - private long _baseAddress; |
1082 | + private volatile long _baseAddress; |
1083 | |
1084 | private final Map<Long, FileChannel> _journalFileChannels = new HashMap<Long, FileChannel>(); |
1085 | |
1086 | @@ -191,6 +192,8 @@ |
1087 | |
1088 | private final TransactionPlayerListener _listener = new ProactiveRollbackListener(); |
1089 | |
1090 | + private final AtomicBoolean _writePagePruning = new AtomicBoolean(true); |
1091 | + |
1092 | private final AtomicBoolean _rollbackPruning = new AtomicBoolean(true); |
1093 | |
1094 | /* |
1095 | @@ -202,8 +205,6 @@ |
1096 | |
1097 | private volatile int _copiesPerCycle = DEFAULT_COPIES_PER_CYCLE; |
1098 | |
1099 | - private volatile int _pageListSizeBase = DEFAULT_PAGE_MAP_SIZE_BASE; |
1100 | - |
1101 | private volatile long _copierTimestampLimit = Long.MAX_VALUE; |
1102 | |
1103 | private volatile long _earliestCommittedTimestamp = Long.MAX_VALUE; |
1104 | @@ -423,6 +424,11 @@ |
1105 | _rollbackPruning.set(rollbackPruning); |
1106 | } |
1107 | |
1108 | + @Override |
1109 | + public void setWritePagePruningEnabled(final boolean writePruning) { |
1110 | + _writePagePruning.set(writePruning); |
1111 | + } |
1112 | + |
1113 | public JournalManager(final Persistit persistit) { |
1114 | _persistit = persistit; |
1115 | } |
1116 | @@ -443,6 +449,11 @@ |
1117 | } |
1118 | |
1119 | @Override |
1120 | + public boolean isWritePagePruningEnabled() { |
1121 | + return _writePagePruning.get(); |
1122 | + } |
1123 | + |
1124 | + @Override |
1125 | public String getJournalFilePath() { |
1126 | return _journalFilePath; |
1127 | } |
1128 | @@ -539,16 +550,35 @@ |
1129 | * @return the JOURNAL_COPIER urgency on a scale of 0 to 10 |
1130 | */ |
1131 | @Override |
1132 | - public synchronized int urgency() { |
1133 | + public int urgency() { |
1134 | if (_copyFast.get()) { |
1135 | return URGENT; |
1136 | } |
1137 | - int urgency = _pageList.size() / _pageListSizeBase; |
1138 | - final int journalFileCount = (int) (_currentAddress / _blockSize - _baseAddress / _blockSize); |
1139 | - if (!_appendOnly.get() && journalFileCount > 1) { |
1140 | - urgency += journalFileCount - 1; |
1141 | - } |
1142 | - return Math.min(urgency, URGENT); |
1143 | + |
1144 | + final int journalFileCount = (int) ((_currentAddress - _baseAddress) / _blockSize); |
1145 | + if (journalFileCount < HALF_URGENT) { |
1146 | + return journalFileCount; |
1147 | + } |
1148 | + return Math.min(URGENT, journalFileCount - HALF_URGENT); |
1149 | + } |
1150 | + |
1151 | + /** |
1152 | + * Introduce delay into an application thread when JOURNAL_COPIER thread is |
1153 | + * behind. The amount of delay depends on the value returned by |
1154 | + * {@link #urgency()}. When that value is {@value #URGENT} then the delay is |
1155 | + * {@value #URGENT_COMMIT_DELAY} milliseconds. |
1156 | + * |
1157 | + * @throws PersistitInterruptedException |
1158 | + */ |
1159 | + public void throttle() throws PersistitInterruptedException { |
1160 | + final int urgency = urgency(); |
1161 | + if (!_appendOnly.get()) { |
1162 | + if (urgency == URGENT) { |
1163 | + Util.sleep(URGENT_COMMIT_DELAY); |
1164 | + } else if (urgency >= ALMOST_URGENT) { |
1165 | + Util.sleep(URGENT_COMMIT_DELAY / 4); |
1166 | + } |
1167 | + } |
1168 | } |
1169 | |
1170 | int handleForVolume(final Volume volume) throws PersistitException { |
1171 | @@ -690,6 +720,29 @@ |
1172 | final ByteBuffer bb = buffer.getByteBuffer(); |
1173 | |
1174 | final Volume volume = buffer.getVolume(); |
1175 | + final PageNode pn = lookupUpPageNode(pageAddress, volume); |
1176 | + if (pn == null) { |
1177 | + return false; |
1178 | + } |
1179 | + bb.position(0); |
1180 | + final long recordPageAddress = readPageBufferFromJournal(pn, bb); |
1181 | + _persistit.getIOMeter().chargeReadPageFromJournal(volume, pageAddress, bufferSize, pn.getJournalAddress(), |
1182 | + buffer.getIndex()); |
1183 | + |
1184 | + if (pageAddress != recordPageAddress) { |
1185 | + throw new CorruptJournalException("Record at " + pn + " is not volume/page " + buffer.toString()); |
1186 | + } |
1187 | + |
1188 | + if (bb.limit() != bufferSize) { |
1189 | + throw new CorruptJournalException("Record at " + pn + " is wrong size: expected/actual=" + bufferSize + "/" |
1190 | + + bb.limit()); |
1191 | + } |
1192 | + _readPageCount++; |
1193 | + buffer.getVolume().getStatistics().bumpReadCounter(); |
1194 | + return true; |
1195 | + } |
1196 | + |
1197 | + PageNode lookupUpPageNode(final long pageAddress, final Volume volume) { |
1198 | PageNode pnLookup = null; |
1199 | synchronized (this) { |
1200 | final Integer volumeHandle = _volumeToHandleMap.get(volume); |
1201 | @@ -699,7 +752,7 @@ |
1202 | } |
1203 | |
1204 | if (pnLookup == null) { |
1205 | - return false; |
1206 | + return null; |
1207 | } |
1208 | |
1209 | final PageNode pn = new PageNode(pnLookup.getVolumeHandle(), pnLookup.getPageAddress(), |
1210 | @@ -714,25 +767,9 @@ |
1211 | * new checkpoints and that keeps the copier from deleting it. |
1212 | */ |
1213 | if (pnLookup.isInvalid()) { |
1214 | - return false; |
1215 | - } |
1216 | - |
1217 | - bb.position(0); |
1218 | - final long recordPageAddress = readPageBufferFromJournal(pn, bb); |
1219 | - _persistit.getIOMeter().chargeReadPageFromJournal(volume, pageAddress, bufferSize, pn.getJournalAddress(), |
1220 | - buffer.getIndex()); |
1221 | - |
1222 | - if (pageAddress != recordPageAddress) { |
1223 | - throw new CorruptJournalException("Record at " + pn + " is not volume/page " + buffer.toString()); |
1224 | - } |
1225 | - |
1226 | - if (bb.limit() != bufferSize) { |
1227 | - throw new CorruptJournalException("Record at " + pn + " is wrong size: expected/actual=" + bufferSize + "/" |
1228 | - + bb.limit()); |
1229 | - } |
1230 | - _readPageCount++; |
1231 | - buffer.getVolume().getStatistics().bumpReadCounter(); |
1232 | - return true; |
1233 | + return null; |
1234 | + } |
1235 | + return pn; |
1236 | } |
1237 | |
1238 | private long readPageBufferFromJournal(final PageNode pn, final ByteBuffer bb) throws PersistitIOException, |
1239 | @@ -974,6 +1011,14 @@ |
1240 | // |
1241 | force(); |
1242 | // |
1243 | + // Make sure all copied pages have been flushed to disk. |
1244 | + // |
1245 | + for (final Volume vol : _volumeToHandleMap.keySet()) { |
1246 | + if (vol.isOpened()) { |
1247 | + vol.getStorage().force(); |
1248 | + } |
1249 | + } |
1250 | + // |
1251 | // Prepare room for CP.OVERHEAD bytes in the journal. If doing so |
1252 | // started a new journal file then there's no need to write another |
1253 | // CP record. |
1254 | @@ -1736,6 +1781,9 @@ |
1255 | * three commit modes: SOFT, HARD and GROUP. The two parameters represent |
1256 | * time intervals in milliseconds. |
1257 | * |
1258 | + * @param flushedTimestamp |
1259 | + * a timestamp taken after the transaction buffer belonging to |
1260 | + * the current transaction has been flushed. |
1261 | * @param leadTime |
1262 | * time interval in milliseconds by which to anticipate I/O |
1263 | * completion; the method will return as soon as the I/O |
1264 | @@ -1750,10 +1798,11 @@ |
1265 | * @throws PersistitInterruptedException |
1266 | */ |
1267 | |
1268 | - void waitForDurability(final long leadTime, final long stallTime) throws PersistitException { |
1269 | + void waitForDurability(final long flushedTimestamp, final long leadTime, final long stallTime) |
1270 | + throws PersistitException { |
1271 | final JournalFlusher flusher = _flusher; |
1272 | if (flusher != null) { |
1273 | - flusher.waitForDurability(leadTime, stallTime); |
1274 | + flusher.waitForDurability(flushedTimestamp, leadTime, stallTime); |
1275 | } else { |
1276 | throw new IllegalStateException("JOURNAL_FLUSHER is not running"); |
1277 | } |
1278 | @@ -1829,6 +1878,10 @@ |
1279 | |
1280 | PageNode _previous; |
1281 | |
1282 | + PageNode(final int volumeHandle, final long pageAddress) { |
1283 | + this(volumeHandle, pageAddress, Long.MIN_VALUE, -1); |
1284 | + } |
1285 | + |
1286 | PageNode(final int volumeHandle, final long pageAddress, final long journalAddress, final long timestamp) { |
1287 | this._volumeHandle = volumeHandle; |
1288 | this._pageAddress = pageAddress; |
1289 | @@ -1950,8 +2003,20 @@ |
1290 | |
1291 | @Override |
1292 | public int compare(final PageNode a, final PageNode b) { |
1293 | - return a.getJournalAddress() > b.getJournalAddress() ? 1 : a.getJournalAddress() < b |
1294 | - .getJournalAddress() ? -1 : 0; |
1295 | + if (!a.isInvalid() && !b.isInvalid()) { |
1296 | + return a.getJournalAddress() > b.getJournalAddress() ? 1 : a.getJournalAddress() < b |
1297 | + .getJournalAddress() ? -1 : 0; |
1298 | + } |
1299 | + if (a.isInvalid() && !b.isInvalid()) { |
1300 | + return -1; |
1301 | + } |
1302 | + if (!a.isInvalid() && b.isInvalid()) { |
1303 | + return 1; |
1304 | + } |
1305 | + if (a._volumeHandle != b._volumeHandle) { |
1306 | + return a._volumeHandle - b._volumeHandle; |
1307 | + } |
1308 | + return a._pageAddress > b._pageAddress ? 1 : a._pageAddress < b._pageAddress ? -1 : 0; |
1309 | } |
1310 | }; |
1311 | |
1312 | @@ -2138,7 +2203,7 @@ |
1313 | int divisor = 1; |
1314 | |
1315 | if (iom.recentCharge() < iom.getQuiescentIOthreshold()) { |
1316 | - divisor = URGENT - HALF_URGENT; |
1317 | + divisor = HALF_URGENT; |
1318 | } else if (urgency > HALF_URGENT) { |
1319 | divisor = urgency - HALF_URGENT; |
1320 | } |
1321 | @@ -2176,16 +2241,16 @@ |
1322 | |
1323 | /** |
1324 | * General method used to wait for durability. {@See |
1325 | - * JournalManager#waitForDurability(long, long)}. |
1326 | + * JournalManager#waitForDurability(long, long, long)}. |
1327 | * |
1328 | * @throws PersistitInterruptedException |
1329 | */ |
1330 | - private void waitForDurability(final long leadTime, final long stallTime) throws PersistitException { |
1331 | + private void waitForDurability(final long flushedTimestamp, final long leadTime, final long stallTime) |
1332 | + throws PersistitException { |
1333 | /* |
1334 | * Commit is known durable once the JOURNAL_FLUSHER thread has |
1335 | - * posted a timestamp value to _endTimestamp larger than this. |
1336 | + * posted an _endTimestamp larger than flushedTimestamp. |
1337 | */ |
1338 | - final long timestamp = _persistit.getTimestampAllocator().getCurrentTimestamp(); |
1339 | final long now = System.nanoTime(); |
1340 | |
1341 | while (true) { |
1342 | @@ -2208,7 +2273,7 @@ |
1343 | endTimestamp = _endTimestamp; |
1344 | startTime = _startTime; |
1345 | endTime = _endTime; |
1346 | - if (timestamp > startTimestamp && startTimestamp > endTimestamp) { |
1347 | + if (flushedTimestamp > startTimestamp && startTimestamp > endTimestamp) { |
1348 | estimatedRemainingIoNanos = Math.max(startTime + _expectedIoTime - now, 0); |
1349 | } |
1350 | if (startTimestamp == _startTimestamp && endTimestamp == _endTimestamp) { |
1351 | @@ -2217,7 +2282,7 @@ |
1352 | Util.spinSleep(); |
1353 | } |
1354 | |
1355 | - if (endTimestamp > timestamp && startTimestamp > timestamp) { |
1356 | + if (endTimestamp > flushedTimestamp && startTimestamp > flushedTimestamp) { |
1357 | /* |
1358 | * Done - commit is fully durable |
1359 | */ |
1360 | @@ -2233,7 +2298,7 @@ |
1361 | } |
1362 | |
1363 | long estimatedNanosToFinish = Math.max(estimatedRemainingIoNanos, 0); |
1364 | - if (startTimestamp < timestamp) { |
1365 | + if (startTimestamp < flushedTimestamp) { |
1366 | estimatedNanosToFinish += remainingSleepNanos + _expectedIoTime; |
1367 | } |
1368 | |
1369 | @@ -2262,7 +2327,7 @@ |
1370 | if (_lock.readLock().tryLock(NS_PER_MS, TimeUnit.NANOSECONDS)) { |
1371 | _lock.readLock().unlock(); |
1372 | } |
1373 | - } catch (InterruptedException e) { |
1374 | + } catch (final InterruptedException e) { |
1375 | throw new PersistitInterruptedException(e); |
1376 | } |
1377 | } |
1378 | @@ -2287,7 +2352,7 @@ |
1379 | _waitLoopsWithNoDelay.incrementAndGet(); |
1380 | } |
1381 | } |
1382 | - if (_lastExceptionTimestamp > timestamp) { |
1383 | + if (_lastExceptionTimestamp > flushedTimestamp) { |
1384 | final Exception e = _lastException; |
1385 | if (e instanceof PersistitException) { |
1386 | throw (PersistitException) e; |
1387 | @@ -2428,6 +2493,8 @@ |
1388 | continue; |
1389 | } |
1390 | pageAddress = readPageBufferFromJournal(stablePageNode, bb); |
1391 | + _persistit.getIOMeter().chargeCopyPageFromJournal(volumeRef, pageAddress, volume.getPageSize(), |
1392 | + stablePageNode.getJournalAddress(), urgency()); |
1393 | } catch (final PersistitException ioe) { |
1394 | _persistit |
1395 | .getAlertMonitor() |
1396 | @@ -2455,8 +2522,6 @@ |
1397 | Volume volume = null; |
1398 | int handle = -1; |
1399 | |
1400 | - final HashSet<Volume> volumes = new HashSet<Volume>(); |
1401 | - |
1402 | for (final Iterator<PageNode> iterator = list.iterator(); iterator.hasNext();) { |
1403 | if (_closed.get() && !_copyFast.get() || _appendOnly.get()) { |
1404 | list.clear(); |
1405 | @@ -2512,15 +2577,11 @@ |
1406 | throw ioe; |
1407 | } |
1408 | |
1409 | - volumes.add(volume); |
1410 | _copiedPageCount++; |
1411 | _persistit.getIOMeter().chargeCopyPageToVolume(volume, pageAddress, volume.getPageSize(), |
1412 | - pageNode.getJournalAddress(), _copyFast.get() ? URGENT : urgency()); |
1413 | + pageNode.getJournalAddress(), urgency()); |
1414 | } |
1415 | |
1416 | - for (final Volume vol : volumes) { |
1417 | - vol.getStorage().force(); |
1418 | - } |
1419 | } |
1420 | |
1421 | private void cleanupForCopy(final List<PageNode> list) throws PersistitException { |
1422 | @@ -2567,6 +2628,7 @@ |
1423 | // Detect first journal address holding a mapped page |
1424 | // required for recovery |
1425 | // |
1426 | + |
1427 | for (final PageNode pageNode : _pageMap.values()) { |
1428 | // |
1429 | // If there are multiple versions, we need to keep |
1430 | @@ -2584,9 +2646,9 @@ |
1431 | // |
1432 | for (final Iterator<TransactionMapItem> iterator = _liveTransactionMap.values().iterator(); iterator |
1433 | .hasNext();) { |
1434 | - final TransactionMapItem ts = iterator.next(); |
1435 | - if (ts.getStartAddress() < recoveryBoundary) { |
1436 | - recoveryBoundary = ts.getStartAddress(); |
1437 | + final TransactionMapItem item = iterator.next(); |
1438 | + if (item.getStartAddress() < recoveryBoundary) { |
1439 | + recoveryBoundary = item.getStartAddress(); |
1440 | } |
1441 | } |
1442 | |
1443 | @@ -2749,7 +2811,6 @@ |
1444 | |
1445 | @Override |
1446 | public void store(final long address, final long timestamp, final Exchange exchange) throws PersistitException { |
1447 | - final TransactionStatus ts = _persistit.getTransactionIndex().getStatus(timestamp); |
1448 | exchange.prune(); |
1449 | } |
1450 | |
1451 | @@ -2817,6 +2878,7 @@ |
1452 | /** |
1453 | * Extend ArrayList to export the removeRange method. |
1454 | */ |
1455 | + @SuppressWarnings("serial") |
1456 | static class RangeRemovingArrayList<T> extends ArrayList<T> { |
1457 | @Override |
1458 | public void removeRange(final int fromIndex, final int toIndex) { |
1459 | |
1460 | === added file 'src/main/java/com/persistit/JournalManagerBench.java' |
1461 | --- src/main/java/com/persistit/JournalManagerBench.java 1970-01-01 00:00:00 +0000 |
1462 | +++ src/main/java/com/persistit/JournalManagerBench.java 2012-08-24 14:04:53 +0000 |
1463 | @@ -0,0 +1,186 @@ |
1464 | +/** |
1465 | + * Copyright © 2012 Akiban Technologies, Inc. All rights reserved. |
1466 | + * |
1467 | + * This program and the accompanying materials are made available |
1468 | + * under the terms of the Eclipse Public License v1.0 which |
1469 | + * accompanies this distribution, and is available at |
1470 | + * http://www.eclipse.org/legal/epl-v10.html |
1471 | + * |
1472 | + * This program may also be available under different license terms. |
1473 | + * For more information, see www.akiban.com or contact licensing@akiban.com. |
1474 | + * |
1475 | + * Contributors: |
1476 | + * Akiban Technologies, Inc. |
1477 | + */ |
1478 | + |
1479 | +package com.persistit; |
1480 | + |
1481 | +import java.io.File; |
1482 | +import java.io.RandomAccessFile; |
1483 | +import java.nio.ByteBuffer; |
1484 | +import java.nio.channels.FileChannel; |
1485 | + |
1486 | +import com.persistit.util.ArgParser; |
1487 | + |
1488 | +/** |
1489 | + * Benchmark for primitive I/O simulating HARD (durable) commit. This code is |
1490 | + * intended to explore two elements: |
1491 | + * |
1492 | + * (1) Pre-extending the journal file (so that FileChannel.force(false) usually |
1493 | + * does not need to write any metadata. |
1494 | + * |
1495 | + * (2) Performing I/O in fixed-length blocks so that to write a some bytes the |
1496 | + * file system does not first need to read existing data from disk. |
1497 | + * |
1498 | + * Parameters |
1499 | + * |
1500 | + * align - smallest unit of I/O (default = 1) datapath - directory in which fake |
1501 | + * journal file will be written (default = /tmp/persistit_test_data) buffersize |
1502 | + * - emulated journal buffer size (default = 64M) |
1503 | + * |
1504 | + * |
1505 | + * @author peter |
1506 | + * |
1507 | + */ |
1508 | +public class JournalManagerBench { |
1509 | + |
1510 | + private final long NS_PER_S = 1000000000L; |
1511 | + private final byte[] NULLS = new byte[65536]; |
1512 | + |
1513 | + private final String[] ARG_TEMPLATE = new String[] { "duration|int:10:10:86400|Duration of test in seconds", |
1514 | + "policy|String:HARD|Commit policy: SOFT, HARD or GROUP", |
1515 | + "datapath|String:/tmp/persistit_test_data|Datapath property", |
1516 | + "buffersize|int:64:1:1024|Emulated journal buffer size in MBytes", |
1517 | + "extension|int:0:0:1024|MBytes by which to extend file when full", |
1518 | + "prealloc|int:0:0:1024|Preallocated file size in MBytes", |
1519 | + "align|int:1:1:65536|Blocking factor for I/O size", |
1520 | + "recsize|int:123:64:65536|Emulated transaction record size" }; |
1521 | + |
1522 | + final ByteBuffer buffer; |
1523 | + final ArgParser ap; |
1524 | + |
1525 | + private File file; |
1526 | + private FileChannel fc; |
1527 | + |
1528 | + private long writeAddress = 0; |
1529 | + private long currentAddress = 0; |
1530 | + |
1531 | + long count = 0; |
1532 | + long minTime = Long.MAX_VALUE; |
1533 | + long maxTime = Long.MIN_VALUE; |
1534 | + |
1535 | + final byte[] bytes = new byte[65536]; |
1536 | + |
1537 | + public static void main(final String[] args) throws Exception { |
1538 | + final JournalManagerBench jmb = new JournalManagerBench(args); |
1539 | + jmb.runTest(); |
1540 | + } |
1541 | + |
1542 | + private JournalManagerBench(final String[] args) throws Exception { |
1543 | + ap = new ArgParser("JournalManagerBench", args, ARG_TEMPLATE).strict(); |
1544 | + buffer = ByteBuffer.allocate(ap.getIntValue("buffersize") * 1024 * 1024); |
1545 | + } |
1546 | + |
1547 | + @SuppressWarnings("resource") |
1548 | + private void runTest() throws Exception { |
1549 | + file = new File(ap.getStringValue("datapath"), "JManBench_TestFile"); |
1550 | + fc = new RandomAccessFile(file, "rw").getChannel(); |
1551 | + preallocateFile(ap.getIntValue("prealloc") * 1024 * 1024); |
1552 | + for (int i = 0; i < bytes.length; i++) { |
1553 | + bytes[i] = (byte) ('-'); |
1554 | + } |
1555 | + final int align = ap.getIntValue("align"); |
1556 | + final long extension = ap.getIntValue("extension") * 1024 * 1024; |
1557 | + |
1558 | + final long start = System.nanoTime(); |
1559 | + final long expires = start + ap.getIntValue("duration") * NS_PER_S; |
1560 | + long now = System.nanoTime(); |
1561 | + |
1562 | + while (now < expires) { |
1563 | + doOneCycle(now - start, align, extension, 100); |
1564 | + final long then = System.nanoTime(); |
1565 | + count++; |
1566 | + minTime = Math.min(minTime, then - now); |
1567 | + maxTime = Math.max(maxTime, then - now); |
1568 | + now = then; |
1569 | + } |
1570 | + final long elapsed = now - start; |
1571 | + System.out.printf("%,d commits took %,dms at a rate of %,d/second minimum time=%,dns maximumTime=%,dns\n", |
1572 | + count, elapsed, (count * NS_PER_S) / elapsed, minTime, maxTime); |
1573 | + |
1574 | + } |
1575 | + |
1576 | + private void preallocateFile(final long size) throws Exception { |
1577 | + if (size > 0 && fc.size() > size) { |
1578 | + System.out.printf("Truncating file %s from %,d to %,d\n", file, fc.size(), size); |
1579 | + fc.truncate(size); |
1580 | + } else if (fc.size() < size) { |
1581 | + System.out.printf("Preallocating file %s to size %,d ", file, size); |
1582 | + while (true) { |
1583 | + long remaining = size - fc.size(); |
1584 | + if (remaining <= 0) { |
1585 | + break; |
1586 | + } |
1587 | + if (remaining > buffer.capacity()) { |
1588 | + remaining = buffer.capacity(); |
1589 | + long unaligned = fc.size() % 16384; |
1590 | + if (unaligned > 0) { |
1591 | + remaining = remaining - (16384 - unaligned); |
1592 | + } |
1593 | + } |
1594 | + buffer.position(0).limit((int) remaining); |
1595 | + fc.write(buffer, fc.size()); |
1596 | + System.out.print("."); |
1597 | + } |
1598 | + fc.force(true); |
1599 | + System.out.println("done"); |
1600 | + } |
1601 | + } |
1602 | + |
1603 | + private void doOneCycle(final long time, final int align, final long extension, final int size) throws Exception { |
1604 | + // Make a fake transaction record |
1605 | + String header = String.format("\nsize=%06d count=%06d time=%012d\n", size, count, time); |
1606 | + byte[] b = header.getBytes(); |
1607 | + System.arraycopy(b, 0, bytes, 0, b.length); |
1608 | + |
1609 | + // Add the record, possibly offset to maintaining alignment |
1610 | + int toRewrite = (int) (currentAddress - writeAddress); |
1611 | + buffer.position(toRewrite); |
1612 | + buffer.put(bytes, 0, size); |
1613 | + boolean extended = false; |
1614 | + |
1615 | + int position = buffer.position(); |
1616 | + |
1617 | + // If extension is needed, add those bytes |
1618 | + long currentSize; |
1619 | + if (extension > 0 && writeAddress + buffer.position() > (currentSize = fc.size())) { |
1620 | + long newSize = currentSize + extension; |
1621 | + if (newSize - writeAddress > buffer.capacity()) { |
1622 | + newSize = writeAddress + buffer.capacity(); |
1623 | + assert newSize > currentSize; |
1624 | + } |
1625 | + int add = (int) (newSize - writeAddress - buffer.position()); |
1626 | + while (add > 0) { |
1627 | + buffer.put(NULLS, 0, Math.min(NULLS.length, add)); |
1628 | + add -= NULLS.length; |
1629 | + } |
1630 | + extended = true; |
1631 | + } |
1632 | + |
1633 | + // Write and force the buffer |
1634 | + |
1635 | + buffer.flip(); |
1636 | + fc.write(buffer, writeAddress); |
1637 | + fc.force(extended); |
1638 | + |
1639 | + // Align the bytes to the beginning of the buffer as needed |
1640 | + |
1641 | + currentAddress = writeAddress + position; |
1642 | + buffer.limit(position); |
1643 | + position = (position / align) * align; |
1644 | + buffer.position(position); |
1645 | + buffer.compact(); |
1646 | + writeAddress += position; |
1647 | + } |
1648 | + |
1649 | +} |
1650 | |
1651 | === modified file 'src/main/java/com/persistit/JournalTool.java' |
1652 | --- src/main/java/com/persistit/JournalTool.java 2012-08-24 14:04:53 +0000 |
1653 | +++ src/main/java/com/persistit/JournalTool.java 2012-08-24 14:04:53 +0000 |
1654 | @@ -688,7 +688,10 @@ |
1655 | } |
1656 | |
1657 | protected void flush() { |
1658 | - write(sb.toString()); |
1659 | + if (sb.length() > 0) { |
1660 | + write(sb.toString()); |
1661 | + sb.setLength(0); |
1662 | + } |
1663 | } |
1664 | |
1665 | protected void write(final String msg) { |
1666 | @@ -908,7 +911,6 @@ |
1667 | long address = from + PM.OVERHEAD; |
1668 | int index = 0; |
1669 | int loaded = 0; |
1670 | - sb.setLength(0); |
1671 | long lastPage = Long.MAX_VALUE; |
1672 | int lastVolumeHandle = Integer.MAX_VALUE; |
1673 | for (int remaining = count; remaining > 0; remaining--) { |
1674 | @@ -930,10 +932,7 @@ |
1675 | final long journalAddress = PM.getEntryJournalAddress(_readBuffer, index); |
1676 | if (_selectedPages.isSelected(pageAddress) && _selectedTimestamps.isSelected(pageTimestamp)) { |
1677 | if (pageAddress != lastPage || volumeHandle != lastVolumeHandle) { |
1678 | - if (sb.length() > 0) { |
1679 | - flush(); |
1680 | - sb.setLength(0); |
1681 | - } |
1682 | + flush(); |
1683 | lastPage = pageAddress; |
1684 | lastVolumeHandle = volumeHandle; |
1685 | appendf("-- %5d:%,12d: ", volumeHandle, pageAddress); |
1686 | @@ -953,7 +952,6 @@ |
1687 | long address = from + TM.OVERHEAD; |
1688 | int index = 0; |
1689 | int loaded = 0; |
1690 | - sb.setLength(0); |
1691 | for (int remaining = count; remaining > 0; remaining--) { |
1692 | if (index == loaded) { |
1693 | read(address, Math.min(_readBuffer.capacity(), remaining * TM.ENTRY_SIZE)); |
1694 | @@ -972,7 +970,6 @@ |
1695 | appendf("-- start %,12d commit %,12d @%,18d %s", startTimestamp, commitTimestamp, journalAddress, |
1696 | isCommitted ? "committed" : "uncommitted"); |
1697 | flush(); |
1698 | - sb.setLength(0); |
1699 | index++; |
1700 | } |
1701 | } |
1702 | |
1703 | === modified file 'src/main/java/com/persistit/MVV.java' |
1704 | --- src/main/java/com/persistit/MVV.java 2012-08-24 14:04:53 +0000 |
1705 | +++ src/main/java/com/persistit/MVV.java 2012-08-24 14:04:53 +0000 |
1706 | @@ -19,6 +19,7 @@ |
1707 | import static com.persistit.Buffer.LONGREC_SIZE; |
1708 | import static com.persistit.Buffer.LONGREC_TYPE; |
1709 | import static com.persistit.TransactionIndex.vh2ts; |
1710 | +import static com.persistit.TransactionStatus.ABORTED; |
1711 | import static com.persistit.TransactionStatus.UNCOMMITTED; |
1712 | |
1713 | import java.util.List; |
1714 | @@ -307,7 +308,7 @@ |
1715 | * simply replaced. |
1716 | */ |
1717 | else { |
1718 | - Debug.$assert0.t(verify(target, targetOffset, targetLength)); |
1719 | + assert verify(target, targetOffset, targetLength); |
1720 | /* |
1721 | * Search for the matching version. |
1722 | */ |
1723 | @@ -600,6 +601,37 @@ |
1724 | return true; |
1725 | } |
1726 | |
1727 | + static boolean verify(final TransactionIndex ti, final byte[] bytes, final int offset, final int length) { |
1728 | + if (!isArrayMVV(bytes, offset, length)) { |
1729 | + /* |
1730 | + * Not an MVV |
1731 | + */ |
1732 | + return true; |
1733 | + } |
1734 | + int from = offset + 1; |
1735 | + final long lastVersion = -1; |
1736 | + while (from < offset + length) { |
1737 | + final int vlength = getLength(bytes, from); |
1738 | + final long version = getVersion(bytes, from); |
1739 | + if (vlength < 0 || from + vlength + LENGTH_PER_VERSION > offset + length) { |
1740 | + return false; |
1741 | + } |
1742 | + if (version < lastVersion) { |
1743 | + try { |
1744 | + final long lastVersionTc = ti.commitStatus(lastVersion, UNCOMMITTED, 0); |
1745 | + assert lastVersionTc == ABORTED; |
1746 | + } catch (final InterruptedException e) { |
1747 | + // ignore |
1748 | + } catch (final TimeoutException e) { |
1749 | + // ignore |
1750 | + } |
1751 | + } |
1752 | + assert version != lastVersion; |
1753 | + from += vlength + LENGTH_PER_VERSION; |
1754 | + } |
1755 | + return true; |
1756 | + } |
1757 | + |
1758 | /** |
1759 | * Search for a known version within a MVV array. If the version is found |
1760 | * within the array, copy the contents out to the target and return the |
1761 | |
1762 | === modified file 'src/main/java/com/persistit/MediatedFileChannel.java' |
1763 | --- src/main/java/com/persistit/MediatedFileChannel.java 2012-08-24 14:04:53 +0000 |
1764 | +++ src/main/java/com/persistit/MediatedFileChannel.java 2012-08-24 14:04:53 +0000 |
1765 | @@ -342,7 +342,7 @@ |
1766 | * |
1767 | * @param channel |
1768 | */ |
1769 | - void setErrorInjectingChannelForTests(final FileChannel channel) { |
1770 | + void injectChannelForTests(final FileChannel channel) { |
1771 | ((TestChannelInjector) channel).setChannel(_channel); |
1772 | _channel = channel; |
1773 | } |
1774 | |
1775 | === modified file 'src/main/java/com/persistit/Persistit.java' |
1776 | --- src/main/java/com/persistit/Persistit.java 2012-08-24 14:04:53 +0000 |
1777 | +++ src/main/java/com/persistit/Persistit.java 2012-08-24 14:04:53 +0000 |
1778 | @@ -403,6 +403,7 @@ |
1779 | |
1780 | private final AtomicBoolean _suspendShutdown = new AtomicBoolean(false); |
1781 | private final AtomicBoolean _suspendUpdates = new AtomicBoolean(false); |
1782 | + private final AtomicBoolean _enableBufferInventory = new AtomicBoolean(false); |
1783 | |
1784 | private UtilControl _localGUI; |
1785 | |
1786 | @@ -588,9 +589,7 @@ |
1787 | initializeVolumes(); |
1788 | startJournal(); |
1789 | startBufferPools(); |
1790 | - if (_configuration.getBufferInventoryPathName() != null) { |
1791 | - warmupBufferPools(); |
1792 | - } |
1793 | + preloadBufferPools(); |
1794 | finishRecovery(); |
1795 | startCheckpointManager(); |
1796 | startTransactionIndexPollTask(); |
1797 | @@ -699,6 +698,7 @@ |
1798 | _defaultSplitPolicy = _configuration.getSplitPolicy(); |
1799 | _defaultJoinPolicy = _configuration.getJoinPolicy(); |
1800 | _defaultCommitPolicy = _configuration.getCommitPolicy(); |
1801 | + _enableBufferInventory.set(_configuration.isBufferInventoryEnabled()); |
1802 | } |
1803 | |
1804 | void startCheckpointManager() { |
1805 | @@ -719,10 +719,24 @@ |
1806 | } |
1807 | } |
1808 | |
1809 | - void warmupBufferPools() throws PersistitException { |
1810 | - final String pathName = _configuration.getBufferInventoryPathName(); |
1811 | - for (final BufferPool pool : _bufferPoolTable.values()) { |
1812 | - pool.warmupBufferPool(pathName, pool.toString()); |
1813 | + void recordBufferPoolInventory() { |
1814 | + final long timestamp = _timestampAllocator.getCurrentTimestamp(); |
1815 | + if (_enableBufferInventory.get()) { |
1816 | + for (final BufferPool pool : _bufferPoolTable.values()) { |
1817 | + try { |
1818 | + pool.recordBufferInventory(timestamp); |
1819 | + } catch (PersistitException e) { |
1820 | + getLogBase().bufferInventoryException.log(e); |
1821 | + } |
1822 | + } |
1823 | + } |
1824 | + } |
1825 | + |
1826 | + void preloadBufferPools() throws PersistitException { |
1827 | + if (_configuration.isBufferPreloadEnabled()) { |
1828 | + for (final BufferPool pool : _bufferPoolTable.values()) { |
1829 | + pool.preloadBufferInventory(); |
1830 | + } |
1831 | } |
1832 | } |
1833 | |
1834 | @@ -974,8 +988,6 @@ |
1835 | * |
1836 | * @throws IllegalStateException |
1837 | */ |
1838 | - |
1839 | - // TODO - why not one pool. |
1840 | public void releaseExchange(final Exchange exchange, final boolean secure) { |
1841 | if (exchange == null) { |
1842 | return; |
1843 | @@ -1638,6 +1650,7 @@ |
1844 | } |
1845 | } |
1846 | } |
1847 | + recordBufferPoolInventory(); |
1848 | |
1849 | /* |
1850 | * The copier is responsible for background pruning of aborted |
1851 | @@ -1899,9 +1912,9 @@ |
1852 | checkFatal(); |
1853 | throw new PersistitClosedException(); |
1854 | } |
1855 | - if (Thread.currentThread().isInterrupted()) { |
1856 | - throw new PersistitInterruptedException(new InterruptedException()); |
1857 | - } |
1858 | + // if (Thread.currentThread().isInterrupted()) { |
1859 | + // throw new PersistitInterruptedException(new InterruptedException()); |
1860 | + // } |
1861 | } |
1862 | |
1863 | void checkFatal() throws FatalErrorException { |
1864 | |
1865 | === modified file 'src/main/java/com/persistit/Transaction.java' |
1866 | --- src/main/java/com/persistit/Transaction.java 2012-08-24 14:04:53 +0000 |
1867 | +++ src/main/java/com/persistit/Transaction.java 2012-08-24 14:04:53 +0000 |
1868 | @@ -854,21 +854,27 @@ |
1869 | sequence(COMMIT_FLUSH_A); |
1870 | _commitTimestamp = _persistit.getTimestampAllocator().updateTimestamp(); |
1871 | sequence(COMMIT_FLUSH_C); |
1872 | + long flushedTimetimestamp = 0; |
1873 | boolean committed = false; |
1874 | try { |
1875 | + |
1876 | if (flushTransactionBuffer(false)) { |
1877 | - _persistit.getJournalManager().waitForDurability( |
1878 | - policy == CommitPolicy.SOFT ? _persistit.getTransactionCommitLeadTime() : 0, |
1879 | - policy == CommitPolicy.GROUP ? _persistit.getTransactionCommitStallTime() : 0); |
1880 | + flushedTimetimestamp = _persistit.getTimestampAllocator().getCurrentTimestamp(); |
1881 | } |
1882 | committed = true; |
1883 | } finally { |
1884 | - |
1885 | _persistit.getTransactionIndex().notifyCompleted(_transactionStatus, |
1886 | committed ? _commitTimestamp : TransactionStatus.ABORTED); |
1887 | _commitCompleted = committed; |
1888 | _rollbackPending = _rollbackCompleted = !committed; |
1889 | } |
1890 | + |
1891 | + _persistit.getJournalManager().throttle(); |
1892 | + if (flushedTimetimestamp != 0) { |
1893 | + _persistit.getJournalManager().waitForDurability(flushedTimetimestamp, |
1894 | + policy == CommitPolicy.SOFT ? _persistit.getTransactionCommitLeadTime() : 0, |
1895 | + policy == CommitPolicy.GROUP ? _persistit.getTransactionCommitStallTime() : 0); |
1896 | + } |
1897 | } |
1898 | } |
1899 | |
1900 | |
1901 | === modified file 'src/main/java/com/persistit/TransactionIndex.java' |
1902 | --- src/main/java/com/persistit/TransactionIndex.java 2012-08-24 14:04:53 +0000 |
1903 | +++ src/main/java/com/persistit/TransactionIndex.java 2012-08-24 14:04:53 +0000 |
1904 | @@ -50,7 +50,7 @@ |
1905 | /** |
1906 | * Interval in milliseconds for updating the active transaction cache |
1907 | */ |
1908 | - final static long POLLING_TASK_INTERVAL = 50; |
1909 | + final static long POLLING_TASK_INTERVAL = 10; |
1910 | |
1911 | /** |
1912 | * Default threshold value for moving long-running transactions to the |
1913 | |
1914 | === modified file 'src/main/java/com/persistit/logging/LogBase.java' |
1915 | --- src/main/java/com/persistit/logging/LogBase.java 2012-08-24 14:04:53 +0000 |
1916 | +++ src/main/java/com/persistit/logging/LogBase.java 2012-08-24 14:04:53 +0000 |
1917 | @@ -244,6 +244,15 @@ |
1918 | @Message("ERROR|Too many journal files %,d") |
1919 | public final LogItem tooManyJournalFilesError = PersistitLogMessage.empty(); |
1920 | |
1921 | + @Message("INFO|Preloading buffer pool inventory recorded at %tc") |
1922 | + public final LogItem bufferInventoryLoad = PersistitLogMessage.empty(); |
1923 | + |
1924 | + @Message("INFO|Preloaded %,d of %,d buffers") |
1925 | + public final LogItem bufferInventoryProgress = PersistitLogMessage.empty(); |
1926 | + |
1927 | + @Message("WARNING|Exception while writing buffer pool inventory %s") |
1928 | + public final LogItem bufferInventoryException = PersistitLogMessage.empty(); |
1929 | + |
1930 | public static String recurring(final String message, final int count, final long duration) { |
1931 | return String.format(RECURRING, message, count, duration); |
1932 | } |
1933 | |
1934 | === modified file 'src/main/java/com/persistit/mxbeans/IOMeterMXBean.java' |
1935 | --- src/main/java/com/persistit/mxbeans/IOMeterMXBean.java 2012-08-24 14:04:53 +0000 |
1936 | +++ src/main/java/com/persistit/mxbeans/IOMeterMXBean.java 2012-08-24 14:04:53 +0000 |
1937 | @@ -59,12 +59,12 @@ |
1938 | * |
1939 | */ |
1940 | public final static String[] OPERATION_NAMES = { "Unknown", "Read page from Volume", "Read page from Journal", |
1941 | - "Copy page from journal to volume", "Write page from Journal", "Write Transaction to Journal", "Other", |
1942 | - "Evict page from pool", "Flush journal", "Get page" }; |
1943 | - |
1944 | - public final static String[] OPERATIONS = { "??", "RV", "RJ", "CC", "WJ", "TJ", "XX", "EV", "FJ", "GP" }; |
1945 | - |
1946 | - public final static String[] SUMMARY_ITEMS = { "CC", "RV", "RJ", "WJ", "EV", "FJ" }; |
1947 | + "Copy page from journal", "Copy page to volume", "Write page from Journal", "Write Transaction to Journal", |
1948 | + "Other", "Evict page from pool", "Flush journal", "Get page" }; |
1949 | + |
1950 | + public final static String[] OPERATIONS = { "??", "RV", "RJ", "CJ", "CV", "WJ", "TJ", "XX", "EV", "FJ", "GP" }; |
1951 | + |
1952 | + public final static String[] SUMMARY_ITEMS = { "CJ", "CV", "RV", "RJ", "WJ", "EV", "FJ" }; |
1953 | |
1954 | /** |
1955 | * @return the quiescentIOthreshold |
1956 | |
1957 | === modified file 'src/main/java/com/persistit/mxbeans/JournalManagerMXBean.java' |
1958 | --- src/main/java/com/persistit/mxbeans/JournalManagerMXBean.java 2012-08-24 14:04:53 +0000 |
1959 | +++ src/main/java/com/persistit/mxbeans/JournalManagerMXBean.java 2012-08-24 14:04:53 +0000 |
1960 | @@ -207,6 +207,12 @@ |
1961 | @Description("True to enable pruning of rolled-back transactions") |
1962 | boolean isRollbackPruningEnabled(); |
1963 | |
1964 | + @Description("True to enable pruning when writing pages to journal") |
1965 | + void setWritePagePruningEnabled(boolean rollbackPruning); |
1966 | + |
1967 | + @Description("True to enable pruning when writing pages to journal") |
1968 | + boolean isWritePagePruningEnabled(); |
1969 | + |
1970 | @Description("Degree of urgency for copying pages: 0-10") |
1971 | int urgency(); |
1972 | |
1973 | |
1974 | === modified file 'src/test/java/com/persistit/AccumulatorMemoryTest.java' |
1975 | --- src/test/java/com/persistit/AccumulatorMemoryTest.java 2012-08-24 14:04:53 +0000 |
1976 | +++ src/test/java/com/persistit/AccumulatorMemoryTest.java 2012-08-24 14:04:53 +0000 |
1977 | @@ -21,8 +21,6 @@ |
1978 | |
1979 | import org.junit.Test; |
1980 | |
1981 | -import com.persistit.unit.PersistitUnitTestCase; |
1982 | - |
1983 | public class AccumulatorMemoryTest extends PersistitUnitTestCase { |
1984 | |
1985 | @Test |
1986 | |
1987 | === modified file 'src/test/java/com/persistit/AccumulatorRecoveryTest.java' |
1988 | --- src/test/java/com/persistit/AccumulatorRecoveryTest.java 2012-08-24 14:04:53 +0000 |
1989 | +++ src/test/java/com/persistit/AccumulatorRecoveryTest.java 2012-08-24 14:04:53 +0000 |
1990 | @@ -31,7 +31,6 @@ |
1991 | import com.persistit.TransactionPlayer.TransactionPlayerListener; |
1992 | import com.persistit.exception.PersistitException; |
1993 | import com.persistit.exception.RollbackException; |
1994 | -import com.persistit.unit.PersistitUnitTestCase; |
1995 | import com.persistit.unit.UnitTestProperties; |
1996 | |
1997 | public class AccumulatorRecoveryTest extends PersistitUnitTestCase { |
1998 | |
1999 | === modified file 'src/test/java/com/persistit/AccumulatorTest.java' |
2000 | --- src/test/java/com/persistit/AccumulatorTest.java 2012-08-24 14:04:53 +0000 |
2001 | +++ src/test/java/com/persistit/AccumulatorTest.java 2012-08-24 14:04:53 +0000 |
2002 | @@ -33,7 +33,6 @@ |
2003 | import com.persistit.exception.PersistitException; |
2004 | import com.persistit.exception.TimeoutException; |
2005 | import com.persistit.unit.ConcurrentUtil; |
2006 | -import com.persistit.unit.PersistitUnitTestCase; |
2007 | import com.persistit.unit.UnitTestProperties; |
2008 | |
2009 | public class AccumulatorTest extends PersistitUnitTestCase { |
2010 | |
2011 | === modified file 'src/test/java/com/persistit/AlertMonitorTest.java' |
2012 | --- src/test/java/com/persistit/AlertMonitorTest.java 2012-08-24 14:04:53 +0000 |
2013 | +++ src/test/java/com/persistit/AlertMonitorTest.java 2012-08-24 14:04:53 +0000 |
2014 | @@ -31,7 +31,6 @@ |
2015 | import com.persistit.logging.PersistitLevel; |
2016 | import com.persistit.logging.PersistitLogMessage.LogItem; |
2017 | import com.persistit.logging.PersistitLogger; |
2018 | -import com.persistit.unit.PersistitUnitTestCase; |
2019 | |
2020 | public class AlertMonitorTest extends PersistitUnitTestCase { |
2021 | |
2022 | |
2023 | === modified file 'src/test/java/com/persistit/BackupTaskTest.java' |
2024 | --- src/test/java/com/persistit/BackupTaskTest.java 2012-08-24 14:04:53 +0000 |
2025 | +++ src/test/java/com/persistit/BackupTaskTest.java 2012-08-24 14:04:53 +0000 |
2026 | @@ -29,7 +29,6 @@ |
2027 | |
2028 | import org.junit.Test; |
2029 | |
2030 | -import com.persistit.unit.PersistitUnitTestCase; |
2031 | import com.persistit.unit.UnitTestProperties; |
2032 | |
2033 | public class BackupTaskTest extends PersistitUnitTestCase { |
2034 | |
2035 | === modified file 'src/test/java/com/persistit/BufferMaxPack.java' |
2036 | --- src/test/java/com/persistit/BufferMaxPack.java 2012-08-24 14:04:53 +0000 |
2037 | +++ src/test/java/com/persistit/BufferMaxPack.java 2012-08-24 14:04:53 +0000 |
2038 | @@ -20,7 +20,6 @@ |
2039 | import org.junit.Test; |
2040 | |
2041 | import com.persistit.ValueHelper.RawValueWriter; |
2042 | -import com.persistit.unit.PersistitUnitTestCase; |
2043 | |
2044 | public class BufferMaxPack extends PersistitUnitTestCase { |
2045 | |
2046 | |
2047 | === modified file 'src/test/java/com/persistit/BufferPoolTest.java' |
2048 | --- src/test/java/com/persistit/BufferPoolTest.java 2012-08-24 14:04:53 +0000 |
2049 | +++ src/test/java/com/persistit/BufferPoolTest.java 2012-08-24 14:04:53 +0000 |
2050 | @@ -26,7 +26,6 @@ |
2051 | import org.junit.Test; |
2052 | |
2053 | import com.persistit.BufferPool.BufferHolder; |
2054 | -import com.persistit.unit.PersistitUnitTestCase; |
2055 | |
2056 | public class BufferPoolTest extends PersistitUnitTestCase { |
2057 | |
2058 | |
2059 | === modified file 'src/test/java/com/persistit/BufferTest.java' |
2060 | --- src/test/java/com/persistit/BufferTest.java 2012-08-24 14:04:53 +0000 |
2061 | +++ src/test/java/com/persistit/BufferTest.java 2012-08-24 14:04:53 +0000 |
2062 | @@ -27,7 +27,6 @@ |
2063 | import com.persistit.exception.RebalanceException; |
2064 | import com.persistit.policy.JoinPolicy; |
2065 | import com.persistit.policy.SplitPolicy; |
2066 | -import com.persistit.unit.PersistitUnitTestCase; |
2067 | |
2068 | public class BufferTest extends PersistitUnitTestCase { |
2069 | int leftn; |
2070 | |
2071 | === modified file 'src/test/java/com/persistit/BufferTest2.java' |
2072 | --- src/test/java/com/persistit/BufferTest2.java 2012-08-24 14:04:53 +0000 |
2073 | +++ src/test/java/com/persistit/BufferTest2.java 2012-08-24 14:04:53 +0000 |
2074 | @@ -33,7 +33,6 @@ |
2075 | import com.persistit.exception.RebalanceException; |
2076 | import com.persistit.policy.JoinPolicy; |
2077 | import com.persistit.policy.SplitPolicy; |
2078 | -import com.persistit.unit.PersistitUnitTestCase; |
2079 | |
2080 | public class BufferTest2 extends PersistitUnitTestCase { |
2081 | |
2082 | |
2083 | === modified file 'src/test/java/com/persistit/Bug1003578Test.java' |
2084 | --- src/test/java/com/persistit/Bug1003578Test.java 2012-08-24 14:04:53 +0000 |
2085 | +++ src/test/java/com/persistit/Bug1003578Test.java 2012-08-24 14:04:53 +0000 |
2086 | @@ -27,7 +27,6 @@ |
2087 | import org.junit.Test; |
2088 | |
2089 | import com.persistit.exception.PersistitException; |
2090 | -import com.persistit.unit.PersistitUnitTestCase; |
2091 | |
2092 | /** |
2093 | * https://bugs.launchpad.net/akiban-persistit/+bug/1003578 |
2094 | |
2095 | === modified file 'src/test/java/com/persistit/Bug1017957Test.java' |
2096 | --- src/test/java/com/persistit/Bug1017957Test.java 2012-08-24 14:04:53 +0000 |
2097 | +++ src/test/java/com/persistit/Bug1017957Test.java 2012-08-24 14:04:53 +0000 |
2098 | @@ -23,7 +23,6 @@ |
2099 | |
2100 | import org.junit.Test; |
2101 | |
2102 | -import com.persistit.unit.PersistitUnitTestCase; |
2103 | import com.persistit.unit.UnitTestProperties; |
2104 | |
2105 | /** |
2106 | |
2107 | === modified file 'src/test/java/com/persistit/Bug1018526Test.java' |
2108 | --- src/test/java/com/persistit/Bug1018526Test.java 2012-08-24 14:04:53 +0000 |
2109 | +++ src/test/java/com/persistit/Bug1018526Test.java 2012-08-24 14:04:53 +0000 |
2110 | @@ -27,7 +27,6 @@ |
2111 | import org.junit.Test; |
2112 | |
2113 | import com.persistit.JournalManager.TreeDescriptor; |
2114 | -import com.persistit.unit.PersistitUnitTestCase; |
2115 | |
2116 | public class Bug1018526Test extends PersistitUnitTestCase { |
2117 | |
2118 | |
2119 | === modified file 'src/test/java/com/persistit/Bug706132Test.java' |
2120 | --- src/test/java/com/persistit/Bug706132Test.java 2012-08-24 14:04:53 +0000 |
2121 | +++ src/test/java/com/persistit/Bug706132Test.java 2012-08-24 14:04:53 +0000 |
2122 | @@ -22,7 +22,6 @@ |
2123 | import org.junit.Test; |
2124 | |
2125 | import com.persistit.policy.SplitPolicy; |
2126 | -import com.persistit.unit.PersistitUnitTestCase; |
2127 | |
2128 | /* |
2129 | * Stress10 with 1K pages exhibited a failure in which a value was simply not |
2130 | |
2131 | === modified file 'src/test/java/com/persistit/Bug708592Test.java' |
2132 | --- src/test/java/com/persistit/Bug708592Test.java 2012-08-24 14:04:53 +0000 |
2133 | +++ src/test/java/com/persistit/Bug708592Test.java 2012-08-24 14:04:53 +0000 |
2134 | @@ -21,7 +21,6 @@ |
2135 | import org.junit.Test; |
2136 | |
2137 | import com.persistit.policy.SplitPolicy; |
2138 | -import com.persistit.unit.PersistitUnitTestCase; |
2139 | |
2140 | /* |
2141 | * Got this while loading sample data. Apparently the new PACK split policy |
2142 | |
2143 | === modified file 'src/test/java/com/persistit/Bug739533Test.java' |
2144 | --- src/test/java/com/persistit/Bug739533Test.java 2012-08-24 14:04:53 +0000 |
2145 | +++ src/test/java/com/persistit/Bug739533Test.java 2012-08-24 14:04:53 +0000 |
2146 | @@ -17,8 +17,6 @@ |
2147 | |
2148 | import org.junit.Test; |
2149 | |
2150 | -import com.persistit.unit.PersistitUnitTestCase; |
2151 | - |
2152 | public class Bug739533Test extends PersistitUnitTestCase { |
2153 | |
2154 | private final String _volumeName = "persistit"; |
2155 | |
2156 | === modified file 'src/test/java/com/persistit/Bug777918Test.java' |
2157 | --- src/test/java/com/persistit/Bug777918Test.java 2012-08-24 14:04:53 +0000 |
2158 | +++ src/test/java/com/persistit/Bug777918Test.java 2012-08-24 14:04:53 +0000 |
2159 | @@ -25,7 +25,6 @@ |
2160 | import com.persistit.exception.MissingThreadException; |
2161 | import com.persistit.exception.PersistitException; |
2162 | import com.persistit.exception.TestException; |
2163 | -import com.persistit.unit.PersistitUnitTestCase; |
2164 | import com.persistit.unit.UnitTestProperties; |
2165 | import com.persistit.util.Util; |
2166 | |
2167 | |
2168 | === modified file 'src/test/java/com/persistit/Bug790709Test.java' |
2169 | --- src/test/java/com/persistit/Bug790709Test.java 2012-08-24 14:04:53 +0000 |
2170 | +++ src/test/java/com/persistit/Bug790709Test.java 2012-08-24 14:04:53 +0000 |
2171 | @@ -20,8 +20,6 @@ |
2172 | |
2173 | import org.junit.Test; |
2174 | |
2175 | -import com.persistit.unit.PersistitUnitTestCase; |
2176 | - |
2177 | /** |
2178 | * Bug 790709 This happened on-site at XXX with the halo release. The server |
2179 | * process was running with assertions enabled. The DELETE statement that failed |
2180 | |
2181 | === modified file 'src/test/java/com/persistit/Bug870352Test.java' |
2182 | --- src/test/java/com/persistit/Bug870352Test.java 2012-08-24 14:04:53 +0000 |
2183 | +++ src/test/java/com/persistit/Bug870352Test.java 2012-08-24 14:04:53 +0000 |
2184 | @@ -19,8 +19,6 @@ |
2185 | |
2186 | import org.junit.Test; |
2187 | |
2188 | -import com.persistit.unit.PersistitUnitTestCase; |
2189 | - |
2190 | public class Bug870352Test extends PersistitUnitTestCase { |
2191 | |
2192 | @Test |
2193 | |
2194 | === modified file 'src/test/java/com/persistit/Bug877656Test.java' |
2195 | --- src/test/java/com/persistit/Bug877656Test.java 2012-08-24 14:04:53 +0000 |
2196 | +++ src/test/java/com/persistit/Bug877656Test.java 2012-08-24 14:04:53 +0000 |
2197 | @@ -19,8 +19,6 @@ |
2198 | |
2199 | import org.junit.Test; |
2200 | |
2201 | -import com.persistit.unit.PersistitUnitTestCase; |
2202 | - |
2203 | public class Bug877656Test extends PersistitUnitTestCase { |
2204 | |
2205 | @Test |
2206 | |
2207 | === modified file 'src/test/java/com/persistit/Bug882219Test.java' |
2208 | --- src/test/java/com/persistit/Bug882219Test.java 2012-08-24 14:04:53 +0000 |
2209 | +++ src/test/java/com/persistit/Bug882219Test.java 2012-08-24 14:04:53 +0000 |
2210 | @@ -26,7 +26,6 @@ |
2211 | import com.persistit.Transaction.CommitPolicy; |
2212 | import com.persistit.exception.PersistitIOException; |
2213 | import com.persistit.exception.PersistitInterruptedException; |
2214 | -import com.persistit.unit.PersistitUnitTestCase; |
2215 | |
2216 | /** |
2217 | * Interrupting a thread while it is performing a read or write closes the |
2218 | |
2219 | === modified file 'src/test/java/com/persistit/Bug885477Test.java' |
2220 | --- src/test/java/com/persistit/Bug885477Test.java 2012-08-24 14:04:53 +0000 |
2221 | +++ src/test/java/com/persistit/Bug885477Test.java 2012-08-24 14:04:53 +0000 |
2222 | @@ -21,8 +21,6 @@ |
2223 | |
2224 | import org.junit.Test; |
2225 | |
2226 | -import com.persistit.unit.PersistitUnitTestCase; |
2227 | - |
2228 | /** |
2229 | * Suppose a persistit tree has keys (1, 10), (1, 20), (2, 30). If an exchange's |
2230 | * key is (1), then traverse(GTEQ, false) should traverse to the first record, |
2231 | |
2232 | === modified file 'src/test/java/com/persistit/Bug889850Test.java' |
2233 | --- src/test/java/com/persistit/Bug889850Test.java 2012-08-24 14:04:53 +0000 |
2234 | +++ src/test/java/com/persistit/Bug889850Test.java 2012-08-24 14:04:53 +0000 |
2235 | @@ -17,8 +17,6 @@ |
2236 | |
2237 | import org.junit.Test; |
2238 | |
2239 | -import com.persistit.unit.PersistitUnitTestCase; |
2240 | - |
2241 | /** |
2242 | * |
2243 | * From akiban-server 0.81: |
2244 | |
2245 | === modified file 'src/test/java/com/persistit/Bug911849Test.java' |
2246 | --- src/test/java/com/persistit/Bug911849Test.java 2012-08-24 14:04:53 +0000 |
2247 | +++ src/test/java/com/persistit/Bug911849Test.java 2012-08-24 14:04:53 +0000 |
2248 | @@ -25,7 +25,6 @@ |
2249 | import org.junit.Test; |
2250 | |
2251 | import com.persistit.exception.RollbackException; |
2252 | -import com.persistit.unit.PersistitUnitTestCase; |
2253 | |
2254 | /** |
2255 | * TableStatusRecoveryIT and RenameTableIT fail intermittently |
2256 | |
2257 | === modified file 'src/test/java/com/persistit/Bug912514Test.java' |
2258 | --- src/test/java/com/persistit/Bug912514Test.java 2012-08-24 14:04:53 +0000 |
2259 | +++ src/test/java/com/persistit/Bug912514Test.java 2012-08-24 14:04:53 +0000 |
2260 | @@ -20,8 +20,6 @@ |
2261 | |
2262 | import org.junit.Test; |
2263 | |
2264 | -import com.persistit.unit.PersistitUnitTestCase; |
2265 | - |
2266 | /** |
2267 | * https://bugs.launchpad.net/akiban-persistit/+bug/912514 |
2268 | * |
2269 | |
2270 | === modified file 'src/test/java/com/persistit/Bug915594Test.java' |
2271 | --- src/test/java/com/persistit/Bug915594Test.java 2012-08-24 14:04:53 +0000 |
2272 | +++ src/test/java/com/persistit/Bug915594Test.java 2012-08-24 14:04:53 +0000 |
2273 | @@ -21,8 +21,6 @@ |
2274 | |
2275 | import org.junit.Test; |
2276 | |
2277 | -import com.persistit.unit.PersistitUnitTestCase; |
2278 | - |
2279 | /** |
2280 | * I did an update that failed due to constraint violation and the transaction |
2281 | * was rolled back. |
2282 | |
2283 | === modified file 'src/test/java/com/persistit/Bug918909Test.java' |
2284 | --- src/test/java/com/persistit/Bug918909Test.java 2012-08-24 14:04:53 +0000 |
2285 | +++ src/test/java/com/persistit/Bug918909Test.java 2012-08-24 14:04:53 +0000 |
2286 | @@ -22,8 +22,6 @@ |
2287 | |
2288 | import org.junit.Test; |
2289 | |
2290 | -import com.persistit.unit.PersistitUnitTestCase; |
2291 | - |
2292 | /** |
2293 | * At a DP site experiencing very low insert rates the JournalManager was |
2294 | * failing to delete obsolete journal files. This was noticed just prior to the |
2295 | |
2296 | === modified file 'src/test/java/com/persistit/Bug920754Test.java' |
2297 | --- src/test/java/com/persistit/Bug920754Test.java 2012-08-24 14:04:53 +0000 |
2298 | +++ src/test/java/com/persistit/Bug920754Test.java 2012-08-24 14:04:53 +0000 |
2299 | @@ -19,8 +19,6 @@ |
2300 | |
2301 | import org.junit.Test; |
2302 | |
2303 | -import com.persistit.unit.PersistitUnitTestCase; |
2304 | - |
2305 | public class Bug920754Test extends PersistitUnitTestCase { |
2306 | /* |
2307 | * https://bugs.launchpad.net/akiban-persistit/+bug/920754 |
2308 | |
2309 | === modified file 'src/test/java/com/persistit/Bug923790Test.java' |
2310 | --- src/test/java/com/persistit/Bug923790Test.java 2012-08-24 14:04:53 +0000 |
2311 | +++ src/test/java/com/persistit/Bug923790Test.java 2012-08-24 14:04:53 +0000 |
2312 | @@ -19,7 +19,6 @@ |
2313 | |
2314 | import org.junit.Test; |
2315 | |
2316 | -import com.persistit.unit.PersistitUnitTestCase; |
2317 | import com.persistit.util.Util; |
2318 | |
2319 | /** |
2320 | |
2321 | === modified file 'src/test/java/com/persistit/Bug927701Test.java' |
2322 | --- src/test/java/com/persistit/Bug927701Test.java 2012-08-24 14:04:53 +0000 |
2323 | +++ src/test/java/com/persistit/Bug927701Test.java 2012-08-24 14:04:53 +0000 |
2324 | @@ -19,8 +19,6 @@ |
2325 | |
2326 | import org.junit.Test; |
2327 | |
2328 | -import com.persistit.unit.PersistitUnitTestCase; |
2329 | - |
2330 | /** |
2331 | * Failure detected during TPCC testing. Upon restarting server, the following |
2332 | * error was emitted: |
2333 | @@ -54,7 +52,7 @@ |
2334 | @Test |
2335 | public void testBug927701() throws Exception { |
2336 | final JournalManager jman = _persistit.getJournalManager(); |
2337 | - _persistit.getCleanupManager().setMinimumPruningDelay(0); |
2338 | + disableBackgroundCleanup(); |
2339 | jman.setCopierInterval(1000); |
2340 | final long blockSize = jman.getBlockSize(); |
2341 | /* |
2342 | |
2343 | === modified file 'src/test/java/com/persistit/Bug932097Test.java' |
2344 | --- src/test/java/com/persistit/Bug932097Test.java 2012-08-02 04:45:28 +0000 |
2345 | +++ src/test/java/com/persistit/Bug932097Test.java 2012-08-24 14:04:53 +0000 |
2346 | @@ -19,8 +19,6 @@ |
2347 | |
2348 | import org.junit.Test; |
2349 | |
2350 | -import com.persistit.unit.PersistitUnitTestCase; |
2351 | - |
2352 | public class Bug932097Test extends PersistitUnitTestCase { |
2353 | |
2354 | @Test |
2355 | |
2356 | === modified file 'src/test/java/com/persistit/Bug937877Test.java' |
2357 | --- src/test/java/com/persistit/Bug937877Test.java 2012-08-24 14:04:53 +0000 |
2358 | +++ src/test/java/com/persistit/Bug937877Test.java 2012-08-24 14:04:53 +0000 |
2359 | @@ -24,7 +24,6 @@ |
2360 | |
2361 | import org.junit.Test; |
2362 | |
2363 | -import com.persistit.unit.PersistitUnitTestCase; |
2364 | import com.persistit.util.ThreadSequencer; |
2365 | |
2366 | public class Bug937877Test extends PersistitUnitTestCase { |
2367 | |
2368 | === modified file 'src/test/java/com/persistit/Bug942669Test.java' |
2369 | --- src/test/java/com/persistit/Bug942669Test.java 2012-08-24 14:04:53 +0000 |
2370 | +++ src/test/java/com/persistit/Bug942669Test.java 2012-08-24 14:04:53 +0000 |
2371 | @@ -26,8 +26,6 @@ |
2372 | import org.junit.Ignore; |
2373 | import org.junit.Test; |
2374 | |
2375 | -import com.persistit.unit.PersistitUnitTestCase; |
2376 | - |
2377 | public class Bug942669Test extends PersistitUnitTestCase { |
2378 | |
2379 | // See https://bugs.launchpad.net/akiban-persistit/+bug/942669 |
2380 | |
2381 | === modified file 'src/test/java/com/persistit/Bug947182Test.java' |
2382 | --- src/test/java/com/persistit/Bug947182Test.java 2012-08-24 14:04:53 +0000 |
2383 | +++ src/test/java/com/persistit/Bug947182Test.java 2012-08-24 14:04:53 +0000 |
2384 | @@ -38,7 +38,6 @@ |
2385 | import org.junit.Test; |
2386 | |
2387 | import com.persistit.exception.PersistitException; |
2388 | -import com.persistit.unit.PersistitUnitTestCase; |
2389 | import com.persistit.unit.UnitTestProperties; |
2390 | |
2391 | public class Bug947182Test extends PersistitUnitTestCase { |
2392 | |
2393 | === modified file 'src/test/java/com/persistit/Bug974589Test.java' |
2394 | --- src/test/java/com/persistit/Bug974589Test.java 2012-08-24 14:04:53 +0000 |
2395 | +++ src/test/java/com/persistit/Bug974589Test.java 2012-08-24 14:04:53 +0000 |
2396 | @@ -22,7 +22,6 @@ |
2397 | import org.junit.Test; |
2398 | |
2399 | import com.persistit.exception.PersistitException; |
2400 | -import com.persistit.unit.PersistitUnitTestCase; |
2401 | import com.persistit.unit.UnitTestProperties; |
2402 | |
2403 | public class Bug974589Test extends PersistitUnitTestCase { |
2404 | |
2405 | === modified file 'src/test/java/com/persistit/Bug980292Test.java' |
2406 | --- src/test/java/com/persistit/Bug980292Test.java 2012-08-24 14:04:53 +0000 |
2407 | +++ src/test/java/com/persistit/Bug980292Test.java 2012-08-24 14:04:53 +0000 |
2408 | @@ -22,7 +22,6 @@ |
2409 | import org.junit.Test; |
2410 | |
2411 | import com.persistit.exception.PersistitException; |
2412 | -import com.persistit.unit.PersistitUnitTestCase; |
2413 | import com.persistit.unit.UnitTestProperties; |
2414 | |
2415 | public class Bug980292Test extends PersistitUnitTestCase { |
2416 | |
2417 | === modified file 'src/test/java/com/persistit/Bug989202Test.java' |
2418 | --- src/test/java/com/persistit/Bug989202Test.java 2012-08-24 14:04:53 +0000 |
2419 | +++ src/test/java/com/persistit/Bug989202Test.java 2012-08-24 14:04:53 +0000 |
2420 | @@ -19,8 +19,6 @@ |
2421 | |
2422 | import org.junit.Test; |
2423 | |
2424 | -import com.persistit.unit.PersistitUnitTestCase; |
2425 | - |
2426 | /** |
2427 | * Persistit rev: 284 |
2428 | * |
2429 | |
2430 | === modified file 'src/test/java/com/persistit/Bug992801Test.java' |
2431 | --- src/test/java/com/persistit/Bug992801Test.java 2012-08-24 14:04:53 +0000 |
2432 | +++ src/test/java/com/persistit/Bug992801Test.java 2012-08-24 14:04:53 +0000 |
2433 | @@ -19,8 +19,6 @@ |
2434 | |
2435 | import org.junit.Test; |
2436 | |
2437 | -import com.persistit.unit.PersistitUnitTestCase; |
2438 | - |
2439 | public class Bug992801Test extends PersistitUnitTestCase { |
2440 | /* |
2441 | * https://bugs.launchpad.net/akiban-persistit/+bug/992801 |
2442 | |
2443 | === modified file 'src/test/java/com/persistit/Bug996241Test.java' |
2444 | --- src/test/java/com/persistit/Bug996241Test.java 2012-08-24 14:04:53 +0000 |
2445 | +++ src/test/java/com/persistit/Bug996241Test.java 2012-08-24 14:04:53 +0000 |
2446 | @@ -20,7 +20,6 @@ |
2447 | import org.junit.Test; |
2448 | |
2449 | import com.persistit.exception.PersistitException; |
2450 | -import com.persistit.unit.PersistitUnitTestCase; |
2451 | import com.persistit.unit.UnitTestProperties; |
2452 | |
2453 | public class Bug996241Test extends PersistitUnitTestCase { |
2454 | |
2455 | === modified file 'src/test/java/com/persistit/ClassIndexTest.java' |
2456 | --- src/test/java/com/persistit/ClassIndexTest.java 2012-08-24 14:04:53 +0000 |
2457 | +++ src/test/java/com/persistit/ClassIndexTest.java 2012-08-24 14:04:53 +0000 |
2458 | @@ -31,7 +31,6 @@ |
2459 | import org.junit.Test; |
2460 | |
2461 | import com.persistit.Transaction.CommitPolicy; |
2462 | -import com.persistit.unit.PersistitUnitTestCase; |
2463 | import com.persistit.util.Util; |
2464 | |
2465 | public class ClassIndexTest extends PersistitUnitTestCase { |
2466 | |
2467 | === modified file 'src/test/java/com/persistit/CleanupManagerTest.java' |
2468 | --- src/test/java/com/persistit/CleanupManagerTest.java 2012-08-24 14:04:53 +0000 |
2469 | +++ src/test/java/com/persistit/CleanupManagerTest.java 2012-08-24 14:04:53 +0000 |
2470 | @@ -24,7 +24,6 @@ |
2471 | |
2472 | import com.persistit.CleanupManager.CleanupAction; |
2473 | import com.persistit.exception.PersistitException; |
2474 | -import com.persistit.unit.PersistitUnitTestCase; |
2475 | |
2476 | public class CleanupManagerTest extends PersistitUnitTestCase { |
2477 | |
2478 | |
2479 | === modified file 'src/test/java/com/persistit/CommandLineTest.java' |
2480 | --- src/test/java/com/persistit/CommandLineTest.java 2012-08-24 14:04:53 +0000 |
2481 | +++ src/test/java/com/persistit/CommandLineTest.java 2012-08-24 14:04:53 +0000 |
2482 | @@ -29,7 +29,6 @@ |
2483 | |
2484 | import org.junit.Test; |
2485 | |
2486 | -import com.persistit.unit.PersistitUnitTestCase; |
2487 | import com.persistit.util.Util; |
2488 | |
2489 | public class CommandLineTest extends PersistitUnitTestCase { |
2490 | |
2491 | === modified file 'src/test/java/com/persistit/ConfigurationTest.java' |
2492 | --- src/test/java/com/persistit/ConfigurationTest.java 2012-08-24 14:04:53 +0000 |
2493 | +++ src/test/java/com/persistit/ConfigurationTest.java 2012-08-24 14:04:53 +0000 |
2494 | @@ -33,7 +33,6 @@ |
2495 | import org.junit.Test; |
2496 | |
2497 | import com.persistit.Configuration.BufferPoolConfiguration; |
2498 | -import com.persistit.unit.PersistitUnitTestCase; |
2499 | |
2500 | public class ConfigurationTest extends PersistitUnitTestCase { |
2501 | |
2502 | |
2503 | === modified file 'src/test/java/com/persistit/CorruptVolumeTest.java' |
2504 | --- src/test/java/com/persistit/CorruptVolumeTest.java 2012-08-24 14:04:53 +0000 |
2505 | +++ src/test/java/com/persistit/CorruptVolumeTest.java 2012-08-24 14:04:53 +0000 |
2506 | @@ -23,7 +23,6 @@ |
2507 | |
2508 | import com.persistit.exception.CorruptVolumeException; |
2509 | import com.persistit.exception.PersistitException; |
2510 | -import com.persistit.unit.PersistitUnitTestCase; |
2511 | |
2512 | public class CorruptVolumeTest extends PersistitUnitTestCase { |
2513 | |
2514 | |
2515 | === modified file 'src/test/java/com/persistit/DumpTaskTest.java' |
2516 | --- src/test/java/com/persistit/DumpTaskTest.java 2012-08-24 14:04:53 +0000 |
2517 | +++ src/test/java/com/persistit/DumpTaskTest.java 2012-08-24 14:04:53 +0000 |
2518 | @@ -32,7 +32,6 @@ |
2519 | import com.persistit.JournalRecord.IV; |
2520 | import com.persistit.JournalRecord.PA; |
2521 | import com.persistit.exception.PersistitException; |
2522 | -import com.persistit.unit.PersistitUnitTestCase; |
2523 | |
2524 | public class DumpTaskTest extends PersistitUnitTestCase { |
2525 | |
2526 | |
2527 | === modified file 'src/test/java/com/persistit/FastIndexTest.java' |
2528 | --- src/test/java/com/persistit/FastIndexTest.java 2012-08-24 14:04:53 +0000 |
2529 | +++ src/test/java/com/persistit/FastIndexTest.java 2012-08-24 14:04:53 +0000 |
2530 | @@ -22,7 +22,6 @@ |
2531 | import org.junit.Test; |
2532 | |
2533 | import com.persistit.ValueHelper.RawValueWriter; |
2534 | -import com.persistit.unit.PersistitUnitTestCase; |
2535 | |
2536 | public class FastIndexTest extends PersistitUnitTestCase { |
2537 | |
2538 | |
2539 | === modified file 'src/test/java/com/persistit/FatalErrorExceptionTest.java' |
2540 | --- src/test/java/com/persistit/FatalErrorExceptionTest.java 2012-08-24 14:04:53 +0000 |
2541 | +++ src/test/java/com/persistit/FatalErrorExceptionTest.java 2012-08-24 14:04:53 +0000 |
2542 | @@ -20,7 +20,6 @@ |
2543 | import org.junit.Test; |
2544 | |
2545 | import com.persistit.Persistit.FatalErrorException; |
2546 | -import com.persistit.unit.PersistitUnitTestCase; |
2547 | |
2548 | public class FatalErrorExceptionTest extends PersistitUnitTestCase { |
2549 | |
2550 | |
2551 | === modified file 'src/test/java/com/persistit/IOFailureTest.java' |
2552 | --- src/test/java/com/persistit/IOFailureTest.java 2012-08-24 14:04:53 +0000 |
2553 | +++ src/test/java/com/persistit/IOFailureTest.java 2012-08-24 14:04:53 +0000 |
2554 | @@ -34,7 +34,6 @@ |
2555 | import com.persistit.exception.CorruptVolumeException; |
2556 | import com.persistit.exception.PersistitException; |
2557 | import com.persistit.exception.PersistitIOException; |
2558 | -import com.persistit.unit.PersistitUnitTestCase; |
2559 | import com.persistit.unit.UnitTestProperties; |
2560 | |
2561 | public class IOFailureTest extends PersistitUnitTestCase { |
2562 | @@ -57,7 +56,7 @@ |
2563 | |
2564 | private ErrorInjectingFileChannel errorInjectingChannel(final FileChannel channel) { |
2565 | final ErrorInjectingFileChannel eimfc = new ErrorInjectingFileChannel(); |
2566 | - ((MediatedFileChannel) channel).setErrorInjectingChannelForTests(eimfc); |
2567 | + ((MediatedFileChannel) channel).injectChannelForTests(eimfc); |
2568 | return eimfc; |
2569 | } |
2570 | |
2571 | |
2572 | === modified file 'src/test/java/com/persistit/IOMeterChargeBenchmark.java' |
2573 | --- src/test/java/com/persistit/IOMeterChargeBenchmark.java 2012-08-24 14:04:53 +0000 |
2574 | +++ src/test/java/com/persistit/IOMeterChargeBenchmark.java 2012-08-24 14:04:53 +0000 |
2575 | @@ -19,8 +19,6 @@ |
2576 | |
2577 | import org.junit.Test; |
2578 | |
2579 | -import com.persistit.unit.PersistitUnitTestCase; |
2580 | - |
2581 | public class IOMeterChargeBenchmark extends PersistitUnitTestCase { |
2582 | |
2583 | private final static long SECOND = 1000000000; |
2584 | |
2585 | === modified file 'src/test/java/com/persistit/InsertSequenceTest.java' |
2586 | --- src/test/java/com/persistit/InsertSequenceTest.java 2012-08-24 14:04:53 +0000 |
2587 | +++ src/test/java/com/persistit/InsertSequenceTest.java 2012-08-24 14:04:53 +0000 |
2588 | @@ -22,7 +22,6 @@ |
2589 | import com.persistit.Exchange.Sequence; |
2590 | import com.persistit.exception.PersistitException; |
2591 | import com.persistit.policy.SplitPolicy; |
2592 | -import com.persistit.unit.PersistitUnitTestCase; |
2593 | |
2594 | public class InsertSequenceTest extends PersistitUnitTestCase { |
2595 | |
2596 | |
2597 | === modified file 'src/test/java/com/persistit/IntegrityCheckTest.java' |
2598 | --- src/test/java/com/persistit/IntegrityCheckTest.java 2012-08-24 14:04:53 +0000 |
2599 | +++ src/test/java/com/persistit/IntegrityCheckTest.java 2012-08-24 14:04:53 +0000 |
2600 | @@ -25,7 +25,6 @@ |
2601 | import org.junit.Test; |
2602 | |
2603 | import com.persistit.exception.PersistitException; |
2604 | -import com.persistit.unit.PersistitUnitTestCase; |
2605 | |
2606 | public class IntegrityCheckTest extends PersistitUnitTestCase { |
2607 | |
2608 | @@ -55,7 +54,7 @@ |
2609 | @Test |
2610 | public void testSimpleMvvTree() throws Exception { |
2611 | final Exchange ex = _persistit.getExchange(_volumeName, "mvv", true); |
2612 | - _persistit.getCleanupManager().setPollInterval(Integer.MAX_VALUE); |
2613 | + disableBackgroundCleanup(); |
2614 | |
2615 | transactionalStore(ex); |
2616 | |
2617 | @@ -76,7 +75,7 @@ |
2618 | @Test |
2619 | public void testBrokenKeySequence() throws Exception { |
2620 | final Exchange ex = _persistit.getExchange(_volumeName, "mvv", true); |
2621 | - _persistit.getCleanupManager().setPollInterval(Integer.MAX_VALUE); |
2622 | + disableBackgroundCleanup(); |
2623 | |
2624 | transactionalStore(ex); |
2625 | |
2626 | @@ -90,7 +89,7 @@ |
2627 | @Test |
2628 | public void testBrokenMVVs() throws Exception { |
2629 | final Exchange ex = _persistit.getExchange(_volumeName, "mvv", true); |
2630 | - _persistit.getCleanupManager().setPollInterval(Integer.MAX_VALUE); |
2631 | + disableBackgroundCleanup(); |
2632 | transactionalStore(ex); |
2633 | corrupt2(ex); |
2634 | final IntegrityCheck icheck = icheck(); |
2635 | @@ -155,6 +154,7 @@ |
2636 | @Test |
2637 | public void testPruneRemovesAbortedTransactionStatus() throws Exception { |
2638 | _persistit.getJournalManager().setRollbackPruningEnabled(false); |
2639 | + _persistit.getJournalManager().setWritePagePruningEnabled(false); |
2640 | |
2641 | for (int i = 0; i < 10; i++) { |
2642 | final Exchange ex = _persistit.getExchange(_volumeName, "mvv" + i, true); |
2643 | @@ -179,9 +179,7 @@ |
2644 | _persistit = new Persistit(); |
2645 | _persistit.getRecoveryManager().setRecoveryDisabledForTestMode(true); |
2646 | _persistit.getJournalManager().setRollbackPruningEnabled(false); |
2647 | - _persistit.getCleanupManager().setMinimumPruningDelay(0); // no |
2648 | - // background |
2649 | - // pruningIn |
2650 | + disableBackgroundCleanup(); |
2651 | _persistit.initialize(properties); |
2652 | |
2653 | for (int i = 0; i < 10; i++) { |
2654 | |
2655 | === modified file 'src/test/java/com/persistit/JournalManagerTest.java' |
2656 | --- src/test/java/com/persistit/JournalManagerTest.java 2012-08-24 14:04:53 +0000 |
2657 | +++ src/test/java/com/persistit/JournalManagerTest.java 2012-08-24 14:04:53 +0000 |
2658 | @@ -47,7 +47,6 @@ |
2659 | import com.persistit.TransactionPlayer.TransactionPlayerListener; |
2660 | import com.persistit.exception.PersistitException; |
2661 | import com.persistit.unit.ConcurrentUtil.ThrowingRunnable; |
2662 | -import com.persistit.unit.PersistitUnitTestCase; |
2663 | import com.persistit.unit.UnitTestProperties; |
2664 | import com.persistit.util.Util; |
2665 | |
2666 | @@ -89,6 +88,7 @@ |
2667 | jman.writeCheckpointToJournal(checkpoint1); |
2668 | final Exchange exchange = _persistit.getExchange(_volumeName, "JournalManagerTest1", false); |
2669 | volume.getTree("JournalManagerTest1", false).resetHandle(); |
2670 | + |
2671 | assertTrue(exchange.next(true)); |
2672 | final long[] timestamps = new long[100]; |
2673 | |
2674 | @@ -299,6 +299,7 @@ |
2675 | public void testRollback() throws Exception { |
2676 | // Allow test to control when pruning will happen |
2677 | _persistit.getJournalManager().setRollbackPruningEnabled(false); |
2678 | + _persistit.getJournalManager().setWritePagePruningEnabled(false); |
2679 | final Transaction txn = _persistit.getTransaction(); |
2680 | for (int i = 0; i < 10; i++) { |
2681 | txn.begin(); |
2682 | @@ -349,6 +350,8 @@ |
2683 | public void testRollbackLongRecords() throws Exception { |
2684 | // Allow test to control when pruning will happen |
2685 | _persistit.getJournalManager().setRollbackPruningEnabled(false); |
2686 | + _persistit.getJournalManager().setWritePagePruningEnabled(false); |
2687 | + |
2688 | final Volume volume = _persistit.getVolume(_volumeName); |
2689 | final Transaction txn = _persistit.getTransaction(); |
2690 | for (int i = 0; i < 10; i++) { |
2691 | |
2692 | === modified file 'src/test/java/com/persistit/KeyHistogramTest.java' |
2693 | --- src/test/java/com/persistit/KeyHistogramTest.java 2012-08-24 14:04:53 +0000 |
2694 | +++ src/test/java/com/persistit/KeyHistogramTest.java 2012-08-24 14:04:53 +0000 |
2695 | @@ -19,8 +19,6 @@ |
2696 | |
2697 | import org.junit.Test; |
2698 | |
2699 | -import com.persistit.unit.PersistitUnitTestCase; |
2700 | - |
2701 | public class KeyHistogramTest extends PersistitUnitTestCase { |
2702 | |
2703 | private final String _volumeName = "persistit"; |
2704 | |
2705 | === modified file 'src/test/java/com/persistit/MVCCPruneBufferTest.java' |
2706 | --- src/test/java/com/persistit/MVCCPruneBufferTest.java 2012-08-24 14:04:53 +0000 |
2707 | +++ src/test/java/com/persistit/MVCCPruneBufferTest.java 2012-08-24 14:04:53 +0000 |
2708 | @@ -144,7 +144,7 @@ |
2709 | |
2710 | @Test |
2711 | public void testPruneLongRecordsSimple() throws Exception { |
2712 | - _persistit.getCleanupManager().setPollInterval(-1); |
2713 | + disableBackgroundCleanup(); |
2714 | trx1.begin(); |
2715 | storeLongMVV(ex1, "x"); |
2716 | trx1.commit(); |
2717 | @@ -156,7 +156,7 @@ |
2718 | |
2719 | @Test |
2720 | public void testPruneLongRecordsWithRollback() throws Exception { |
2721 | - _persistit.getCleanupManager().setPollInterval(-1); |
2722 | + disableBackgroundCleanup(); |
2723 | /* |
2724 | * Start a concurrent transaction to prevent pruning during the store |
2725 | * operations. |
2726 | @@ -182,7 +182,7 @@ |
2727 | |
2728 | @Test |
2729 | public void induceBug1006576() throws Exception { |
2730 | - _persistit.getCleanupManager().setPollInterval(-1); |
2731 | + disableBackgroundCleanup(); |
2732 | trx1.begin(); |
2733 | storeLongMVV(ex1, "x"); |
2734 | storeLongMVV(ex1, "y"); |
2735 | @@ -203,7 +203,7 @@ |
2736 | |
2737 | @Test |
2738 | public void induceBug1005206() throws Exception { |
2739 | - _persistit.getCleanupManager().setPollInterval(-1); |
2740 | + disableBackgroundCleanup(); |
2741 | trx1.begin(); |
2742 | storeLongMVV(ex1, "x"); |
2743 | |
2744 | @@ -297,6 +297,27 @@ |
2745 | } |
2746 | } |
2747 | |
2748 | + @Test |
2749 | + public void testWritePagePrune() throws Exception { |
2750 | + final Transaction txn = _persistit.getTransaction(); |
2751 | + try { |
2752 | + txn.begin(); |
2753 | + storeNewVersion(1); |
2754 | + txn.commit(); |
2755 | + } finally { |
2756 | + txn.end(); |
2757 | + } |
2758 | + final Volume volume = _persistit.getVolume(TEST_VOLUME_NAME); |
2759 | + final Buffer buffer = volume.getPool().get(volume, 3, true, true); |
2760 | + assertTrue("Should have multiple MVV records", buffer.getMvvCount() > 2); |
2761 | + _persistit.getJournalManager().setWritePagePruningEnabled(true); |
2762 | + _persistit.getTransactionIndex().updateActiveTransactionCache(); |
2763 | + buffer.setDirtyAtTimestamp(_persistit.getCurrentTimestamp()); |
2764 | + buffer.writePage(); |
2765 | + assertTrue("Should no more than one MVV record", buffer.getMvvCount() < 2); |
2766 | + buffer.release(); |
2767 | + } |
2768 | + |
2769 | private void storeNewVersion(final int cycle) throws Exception { |
2770 | final Exchange exchange = _persistit.getExchange(TEST_VOLUME_NAME, |
2771 | String.format("%s%04d", TEST_TREE_NAME, cycle), true); |
2772 | |
2773 | === modified file 'src/test/java/com/persistit/MVCCTestBase.java' |
2774 | --- src/test/java/com/persistit/MVCCTestBase.java 2012-08-24 14:04:53 +0000 |
2775 | +++ src/test/java/com/persistit/MVCCTestBase.java 2012-08-24 14:04:53 +0000 |
2776 | @@ -24,7 +24,6 @@ |
2777 | |
2778 | import com.persistit.exception.PersistitException; |
2779 | import com.persistit.exception.PersistitInterruptedException; |
2780 | -import com.persistit.unit.PersistitUnitTestCase; |
2781 | import com.persistit.util.Util; |
2782 | |
2783 | public abstract class MVCCTestBase extends PersistitUnitTestCase { |
2784 | |
2785 | === added file 'src/test/java/com/persistit/PersistitUnitTestCase.java' |
2786 | --- src/test/java/com/persistit/PersistitUnitTestCase.java 1970-01-01 00:00:00 +0000 |
2787 | +++ src/test/java/com/persistit/PersistitUnitTestCase.java 2012-08-24 14:04:53 +0000 |
2788 | @@ -0,0 +1,136 @@ |
2789 | +/** |
2790 | + * Copyright © 2005-2012 Akiban Technologies, Inc. All rights reserved. |
2791 | + * |
2792 | + * This program and the accompanying materials are made available |
2793 | + * under the terms of the Eclipse Public License v1.0 which |
2794 | + * accompanies this distribution, and is available at |
2795 | + * http://www.eclipse.org/legal/epl-v10.html |
2796 | + * |
2797 | + * This program may also be available under different license terms. |
2798 | + * For more information, see www.akiban.com or contact licensing@akiban.com. |
2799 | + * |
2800 | + * Contributors: |
2801 | + * Akiban Technologies, Inc. |
2802 | + */ |
2803 | + |
2804 | +package com.persistit; |
2805 | + |
2806 | +import java.lang.ref.WeakReference; |
2807 | +import java.util.Map; |
2808 | +import java.util.Properties; |
2809 | + |
2810 | +import org.junit.After; |
2811 | +import org.junit.Before; |
2812 | + |
2813 | +import com.persistit.exception.PersistitException; |
2814 | +import com.persistit.unit.UnitTestProperties; |
2815 | + |
2816 | +public abstract class PersistitUnitTestCase { |
2817 | + |
2818 | + private final static long TEN_SECONDS = 10L * 1000L * 1000L * 1000L; |
2819 | + |
2820 | + protected final static String RED_FOX = "The quick red fox jumped over the lazy brown dog."; |
2821 | + |
2822 | + protected static String createString(final int exactLength) { |
2823 | + final StringBuilder sb = new StringBuilder(exactLength); |
2824 | + // Simple 0..9a..z string |
2825 | + for (int i = 0; i < 36; ++i) { |
2826 | + sb.append(Character.forDigit(i, 36)); |
2827 | + } |
2828 | + final String numAndLetters = sb.toString(); |
2829 | + while (sb.length() < exactLength) { |
2830 | + sb.append(numAndLetters); |
2831 | + } |
2832 | + return sb.toString().substring(0, exactLength); |
2833 | + } |
2834 | + |
2835 | + protected Persistit _persistit = new Persistit(); |
2836 | + |
2837 | + protected Properties getProperties(final boolean cleanup) { |
2838 | + return UnitTestProperties.getProperties(cleanup); |
2839 | + } |
2840 | + |
2841 | + @Before |
2842 | + public void setUp() throws Exception { |
2843 | + checkNoPersistitThreads(); |
2844 | + _persistit.initialize(getProperties(true)); |
2845 | + } |
2846 | + |
2847 | + @After |
2848 | + public void tearDown() throws Exception { |
2849 | + final WeakReference<Persistit> ref = new WeakReference<Persistit>(_persistit); |
2850 | + _persistit.close(false); |
2851 | + _persistit = null; |
2852 | + |
2853 | + if (!doesRefBecomeNull(ref)) { |
2854 | + System.out.println("Persistit has a leftover strong reference"); |
2855 | + } |
2856 | + checkNoPersistitThreads(); |
2857 | + } |
2858 | + |
2859 | + public void runAllTests() throws Exception { |
2860 | + |
2861 | + } |
2862 | + |
2863 | + public void setPersistit(final Persistit persistit) { |
2864 | + _persistit = persistit; |
2865 | + } |
2866 | + |
2867 | + protected void initAndRunTest() throws Exception { |
2868 | + setUp(); |
2869 | + try { |
2870 | + runAllTests(); |
2871 | + } catch (final Throwable t) { |
2872 | + t.printStackTrace(); |
2873 | + } finally { |
2874 | + tearDown(); |
2875 | + } |
2876 | + } |
2877 | + |
2878 | + private final static String[] PERSISTIT_THREAD_NAMES = { "CHECKPOINT_WRITER", "JOURNAL_COPIER", "JOURNAL_FLUSHER", |
2879 | + "PAGE_WRITER", "TXN_UPDATE" }; |
2880 | + |
2881 | + protected boolean checkNoPersistitThreads() { |
2882 | + boolean alive = false; |
2883 | + final Map<Thread, StackTraceElement[]> map = Thread.getAllStackTraces(); |
2884 | + for (final Thread t : map.keySet()) { |
2885 | + final String name = t.getName(); |
2886 | + for (final String p : PERSISTIT_THREAD_NAMES) { |
2887 | + if (name.contains(p)) { |
2888 | + alive = true; |
2889 | + System.err.println("Thread " + t + " is still alive"); |
2890 | + } |
2891 | + } |
2892 | + } |
2893 | + return alive; |
2894 | + } |
2895 | + |
2896 | + protected void safeCrashAndRestoreProperties() throws PersistitException { |
2897 | + final Properties properties = _persistit.getProperties(); |
2898 | + _persistit.flush(); |
2899 | + _persistit.crash(); |
2900 | + _persistit = new Persistit(); |
2901 | + _persistit.initialize(properties); |
2902 | + } |
2903 | + |
2904 | + protected void crashWithoutFlushAndRestoreProperties() throws PersistitException { |
2905 | + final Properties properties = _persistit.getProperties(); |
2906 | + _persistit.crash(); |
2907 | + _persistit = new Persistit(); |
2908 | + _persistit.initialize(properties); |
2909 | + } |
2910 | + |
2911 | + public static boolean doesRefBecomeNull(final WeakReference<?> ref) throws InterruptedException { |
2912 | + final long expires = System.nanoTime() + TEN_SECONDS; |
2913 | + while (ref.get() != null && System.nanoTime() < expires) { |
2914 | + System.gc(); |
2915 | + Thread.sleep(10); |
2916 | + } |
2917 | + return ref.get() == null; |
2918 | + } |
2919 | + |
2920 | + protected void disableBackgroundCleanup() { |
2921 | + _persistit.getCleanupManager().setPollInterval(-1); |
2922 | + _persistit.getJournalManager().setWritePagePruningEnabled(false); |
2923 | + } |
2924 | +} |
2925 | |
2926 | === modified file 'src/test/java/com/persistit/RecoveryTest.java' |
2927 | --- src/test/java/com/persistit/RecoveryTest.java 2012-08-24 14:04:53 +0000 |
2928 | +++ src/test/java/com/persistit/RecoveryTest.java 2012-08-24 14:04:53 +0000 |
2929 | @@ -37,7 +37,6 @@ |
2930 | import com.persistit.exception.PersistitException; |
2931 | import com.persistit.exception.RollbackException; |
2932 | import com.persistit.exception.TransactionFailedException; |
2933 | -import com.persistit.unit.PersistitUnitTestCase; |
2934 | import com.persistit.unit.UnitTestProperties; |
2935 | |
2936 | public class RecoveryTest extends PersistitUnitTestCase { |
2937 | @@ -53,6 +52,8 @@ |
2938 | public void setUp() throws Exception { |
2939 | super.setUp(); |
2940 | _persistit.getJournalManager().setRollbackPruningEnabled(false); |
2941 | + _persistit.getJournalManager().setWritePagePruningEnabled(false); |
2942 | + |
2943 | } |
2944 | |
2945 | @Override |
2946 | |
2947 | === modified file 'src/test/java/com/persistit/SplitPolicyTest.java' |
2948 | --- src/test/java/com/persistit/SplitPolicyTest.java 2012-08-24 14:04:53 +0000 |
2949 | +++ src/test/java/com/persistit/SplitPolicyTest.java 2012-08-24 14:04:53 +0000 |
2950 | @@ -24,7 +24,6 @@ |
2951 | |
2952 | import com.persistit.Exchange.Sequence; |
2953 | import com.persistit.policy.SplitPolicy; |
2954 | -import com.persistit.unit.PersistitUnitTestCase; |
2955 | |
2956 | public class SplitPolicyTest extends PersistitUnitTestCase { |
2957 | |
2958 | |
2959 | === modified file 'src/test/java/com/persistit/StatisticsTaskTest.java' |
2960 | --- src/test/java/com/persistit/StatisticsTaskTest.java 2012-08-02 04:45:28 +0000 |
2961 | +++ src/test/java/com/persistit/StatisticsTaskTest.java 2012-08-24 14:04:53 +0000 |
2962 | @@ -27,7 +27,6 @@ |
2963 | |
2964 | import com.persistit.StatisticsTask.Display; |
2965 | import com.persistit.StatisticsTask.Stat; |
2966 | -import com.persistit.unit.PersistitUnitTestCase; |
2967 | |
2968 | public class StatisticsTaskTest extends PersistitUnitTestCase { |
2969 | |
2970 | |
2971 | === modified file 'src/test/java/com/persistit/StressRunner.java' |
2972 | --- src/test/java/com/persistit/StressRunner.java 2012-08-24 14:04:53 +0000 |
2973 | +++ src/test/java/com/persistit/StressRunner.java 2012-08-24 14:04:53 +0000 |
2974 | @@ -31,6 +31,7 @@ |
2975 | import com.persistit.stress.MixtureTxn1; |
2976 | import com.persistit.stress.MixtureTxn2; |
2977 | import com.persistit.stress.PersistitMap1; |
2978 | +import com.persistit.stress.PreloadMixtureTxn1; |
2979 | import com.persistit.stress.StartStop; |
2980 | import com.persistit.stress.Stress10Suite; |
2981 | import com.persistit.stress.Stress12txnSuite; |
2982 | @@ -79,6 +80,7 @@ |
2983 | _classes.add(Stress12txnSuite.class); |
2984 | _classes.add(Stress4Suite.class); |
2985 | _classes.add(Stress8txnSuite.class); |
2986 | + _classes.add(PreloadMixtureTxn1.class); |
2987 | } |
2988 | |
2989 | private final static String DURATION_PARAM = "duration="; |
2990 | |
2991 | === added file 'src/test/java/com/persistit/TrackingFileChannel.java' |
2992 | --- src/test/java/com/persistit/TrackingFileChannel.java 1970-01-01 00:00:00 +0000 |
2993 | +++ src/test/java/com/persistit/TrackingFileChannel.java 2012-08-24 14:04:53 +0000 |
2994 | @@ -0,0 +1,175 @@ |
2995 | +/** |
2996 | + * Copyright © 2012 Akiban Technologies, Inc. All rights reserved. |
2997 | + * |
2998 | + * This program and the accompanying materials are made available |
2999 | + * under the terms of the Eclipse Public License v1.0 which |
3000 | + * accompanies this distribution, and is available at |
3001 | + * http://www.eclipse.org/legal/epl-v10.html |
3002 | + * |
3003 | + * This program may also be available under different license terms. |
3004 | + * For more information, see www.akiban.com or contact licensing@akiban.com. |
3005 | + * |
3006 | + * Contributors: |
3007 | + * Akiban Technologies, Inc. |
3008 | + */ |
3009 | + |
3010 | +package com.persistit; |
3011 | + |
3012 | +import static org.junit.Assert.assertEquals; |
3013 | +import static org.junit.Assert.assertTrue; |
3014 | +import static org.junit.Assert.fail; |
3015 | +import java.io.IOException; |
3016 | +import java.nio.ByteBuffer; |
3017 | +import java.nio.MappedByteBuffer; |
3018 | +import java.nio.channels.FileChannel; |
3019 | +import java.nio.channels.FileLock; |
3020 | +import java.nio.channels.ReadableByteChannel; |
3021 | +import java.nio.channels.WritableByteChannel; |
3022 | +import java.util.ArrayList; |
3023 | +import java.util.List; |
3024 | + |
3025 | +import com.persistit.MediatedFileChannel.TestChannelInjector; |
3026 | + |
3027 | +/** |
3028 | + * <p> |
3029 | + * A {@link FileChannel} implementation that simulates IOExceptions under |
3030 | + * control of a unit test program. This class implements only those methods used |
3031 | + * by Persistit; many methods of FileChannel throw |
3032 | + * {@link UnsupportedOperationException}. |
3033 | + * </p> |
3034 | + * |
3035 | + * @author peter |
3036 | + * |
3037 | + */ |
3038 | +class TrackingFileChannel extends FileChannel implements TestChannelInjector { |
3039 | + |
3040 | + volatile FileChannel _channel; |
3041 | + |
3042 | + final List<Long> _writePositions = new ArrayList<Long>(); |
3043 | + |
3044 | + final List<Long> _readPositions = new ArrayList<Long>(); |
3045 | + |
3046 | + @Override |
3047 | + public void setChannel(final FileChannel channel) { |
3048 | + _channel = channel; |
3049 | + } |
3050 | + |
3051 | + @Override |
3052 | + protected void implCloseChannel() throws IOException { |
3053 | + _channel.close(); |
3054 | + } |
3055 | + |
3056 | + @Override |
3057 | + public void force(final boolean metaData) throws IOException { |
3058 | + _channel.force(metaData); |
3059 | + } |
3060 | + |
3061 | + @Override |
3062 | + public int read(final ByteBuffer byteBuffer, final long position) throws IOException { |
3063 | + _readPositions.add(position); |
3064 | + return _channel.read(byteBuffer, position); |
3065 | + } |
3066 | + |
3067 | + @Override |
3068 | + public long size() throws IOException { |
3069 | + return _channel.size(); |
3070 | + } |
3071 | + |
3072 | + @Override |
3073 | + public FileChannel truncate(final long size) throws IOException { |
3074 | + return _channel.truncate(size); |
3075 | + } |
3076 | + |
3077 | + @Override |
3078 | + public synchronized FileLock tryLock(final long position, final long size, final boolean shared) throws IOException { |
3079 | + return _channel.tryLock(position, size, shared); |
3080 | + } |
3081 | + |
3082 | + @Override |
3083 | + public int write(final ByteBuffer byteBuffer, final long position) throws IOException { |
3084 | + _writePositions.add(position); |
3085 | + final int written = _channel.write(byteBuffer, position); |
3086 | + return written; |
3087 | + } |
3088 | + |
3089 | + /* |
3090 | + * -------------------------------- |
3091 | + * |
3092 | + * Persistit does not use these methods and so they are Unsupported. Note |
3093 | + * that it would be difficult to support the relative read/write methods |
3094 | + * because the channel size is unavailable after it is closed. Therefore a |
3095 | + * client of this class must maintain its own position counter and cannot |
3096 | + * use the relative-addressing calls. |
3097 | + * |
3098 | + * -------------------------------- |
3099 | + */ |
3100 | + @Override |
3101 | + public FileLock lock(final long position, final long size, final boolean shared) throws IOException { |
3102 | + throw new UnsupportedOperationException(); |
3103 | + } |
3104 | + |
3105 | + @Override |
3106 | + public MappedByteBuffer map(final MapMode arg0, final long arg1, final long arg2) throws IOException { |
3107 | + throw new UnsupportedOperationException(); |
3108 | + } |
3109 | + |
3110 | + @Override |
3111 | + public long position() throws IOException { |
3112 | + throw new UnsupportedOperationException(); |
3113 | + } |
3114 | + |
3115 | + @Override |
3116 | + public FileChannel position(final long arg0) throws IOException { |
3117 | + throw new UnsupportedOperationException(); |
3118 | + } |
3119 | + |
3120 | + @Override |
3121 | + public int read(final ByteBuffer byteBuffer) throws IOException { |
3122 | + throw new UnsupportedOperationException(); |
3123 | + } |
3124 | + |
3125 | + @Override |
3126 | + public long read(final ByteBuffer[] arg0, final int arg1, final int arg2) throws IOException { |
3127 | + throw new UnsupportedOperationException(); |
3128 | + } |
3129 | + |
3130 | + @Override |
3131 | + public long transferFrom(final ReadableByteChannel arg0, final long arg1, final long arg2) throws IOException { |
3132 | + throw new UnsupportedOperationException(); |
3133 | + } |
3134 | + |
3135 | + @Override |
3136 | + public long transferTo(final long arg0, final long arg1, final WritableByteChannel arg2) throws IOException { |
3137 | + throw new UnsupportedOperationException(); |
3138 | + } |
3139 | + |
3140 | + @Override |
3141 | + public int write(final ByteBuffer byteBuffer) throws IOException { |
3142 | + throw new UnsupportedOperationException(); |
3143 | + } |
3144 | + |
3145 | + @Override |
3146 | + public long write(final ByteBuffer[] arg0, final int arg1, final int arg2) throws IOException { |
3147 | + throw new UnsupportedOperationException(); |
3148 | + } |
3149 | + |
3150 | + public List<Long> getWritePositionList() { |
3151 | + return _writePositions; |
3152 | + } |
3153 | + |
3154 | + public List<Long> getReadPositionList() { |
3155 | + return _readPositions; |
3156 | + } |
3157 | + |
3158 | + public void assertSequential(boolean read, boolean forward) { |
3159 | + final List<Long> list = read ? _readPositions : _writePositions; |
3160 | + long previous = forward ? -1 : Long.MAX_VALUE; |
3161 | + for (final Long position : list) { |
3162 | + if (forward) { |
3163 | + assertTrue("Position should be larger", position > previous); |
3164 | + } else { |
3165 | + assertTrue("Position should be smaller", position < previous); |
3166 | + } |
3167 | + } |
3168 | + } |
3169 | +} |
3170 | |
3171 | === modified file 'src/test/java/com/persistit/TransactionLifetimeTest.java' |
3172 | --- src/test/java/com/persistit/TransactionLifetimeTest.java 2012-08-24 14:04:53 +0000 |
3173 | +++ src/test/java/com/persistit/TransactionLifetimeTest.java 2012-08-24 14:04:53 +0000 |
3174 | @@ -24,7 +24,6 @@ |
3175 | |
3176 | import com.persistit.exception.PersistitException; |
3177 | import com.persistit.exception.TreeNotFoundException; |
3178 | -import com.persistit.unit.PersistitUnitTestCase; |
3179 | |
3180 | /** |
3181 | * Attempt to cover all cases from the pseudo graph below and ensure that the |
3182 | @@ -63,6 +62,8 @@ |
3183 | _persistit.getCheckpointManager().setCheckpointIntervalNanos(FIVE_MIN_NANOS); |
3184 | super.setUp(); |
3185 | _persistit.getJournalManager().setRollbackPruningEnabled(false); |
3186 | + _persistit.getJournalManager().setWritePagePruningEnabled(false); |
3187 | + |
3188 | } |
3189 | |
3190 | @Test |
3191 | @@ -354,6 +355,8 @@ |
3192 | currentInLiveMap = false; |
3193 | safeCrashAndRestoreProperties(); |
3194 | _persistit.getJournalManager().setRollbackPruningEnabled(false); |
3195 | + _persistit.getJournalManager().setWritePagePruningEnabled(false); |
3196 | + |
3197 | } else { |
3198 | fail("Unknown test node: " + curNode); |
3199 | } |
3200 | |
3201 | === modified file 'src/test/java/com/persistit/TransactionTest2.java' |
3202 | --- src/test/java/com/persistit/TransactionTest2.java 2012-08-24 14:04:53 +0000 |
3203 | +++ src/test/java/com/persistit/TransactionTest2.java 2012-08-24 14:04:53 +0000 |
3204 | @@ -33,7 +33,6 @@ |
3205 | import com.persistit.exception.PersistitIOException; |
3206 | import com.persistit.exception.PersistitInterruptedException; |
3207 | import com.persistit.exception.RollbackException; |
3208 | -import com.persistit.unit.PersistitUnitTestCase; |
3209 | |
3210 | /** |
3211 | * Demonstrates the use of Persistit Transactions. This demo runs multiple |
3212 | |
3213 | === modified file 'src/test/java/com/persistit/TreeLifetimeTest.java' |
3214 | --- src/test/java/com/persistit/TreeLifetimeTest.java 2012-08-24 14:04:53 +0000 |
3215 | +++ src/test/java/com/persistit/TreeLifetimeTest.java 2012-08-24 14:04:53 +0000 |
3216 | @@ -45,7 +45,6 @@ |
3217 | import com.persistit.JournalManager.TreeDescriptor; |
3218 | import com.persistit.exception.PersistitException; |
3219 | import com.persistit.exception.TreeNotFoundException; |
3220 | -import com.persistit.unit.PersistitUnitTestCase; |
3221 | import com.persistit.unit.UnitTestProperties; |
3222 | |
3223 | public class TreeLifetimeTest extends PersistitUnitTestCase { |
3224 | |
3225 | === modified file 'src/test/java/com/persistit/TreeTest2.java' |
3226 | --- src/test/java/com/persistit/TreeTest2.java 2012-08-24 14:04:53 +0000 |
3227 | +++ src/test/java/com/persistit/TreeTest2.java 2012-08-24 14:04:53 +0000 |
3228 | @@ -23,8 +23,6 @@ |
3229 | |
3230 | import org.junit.Test; |
3231 | |
3232 | -import com.persistit.unit.PersistitUnitTestCase; |
3233 | - |
3234 | public class TreeTest2 extends PersistitUnitTestCase { |
3235 | |
3236 | final static int COUNT = 10000; |
3237 | |
3238 | === modified file 'src/test/java/com/persistit/ValueTest7.java' |
3239 | --- src/test/java/com/persistit/ValueTest7.java 2012-08-24 14:04:53 +0000 |
3240 | +++ src/test/java/com/persistit/ValueTest7.java 2012-08-24 14:04:53 +0000 |
3241 | @@ -23,7 +23,6 @@ |
3242 | import org.junit.Test; |
3243 | |
3244 | import com.persistit.Value.Version; |
3245 | -import com.persistit.unit.PersistitUnitTestCase; |
3246 | |
3247 | public class ValueTest7 extends PersistitUnitTestCase { |
3248 | |
3249 | |
3250 | === modified file 'src/test/java/com/persistit/VolumeTest.java' |
3251 | --- src/test/java/com/persistit/VolumeTest.java 2012-08-24 14:04:53 +0000 |
3252 | +++ src/test/java/com/persistit/VolumeTest.java 2012-08-24 14:04:53 +0000 |
3253 | @@ -32,7 +32,6 @@ |
3254 | import com.persistit.exception.InUseException; |
3255 | import com.persistit.exception.InvalidVolumeSpecificationException; |
3256 | import com.persistit.exception.VolumeFullException; |
3257 | -import com.persistit.unit.PersistitUnitTestCase; |
3258 | import com.persistit.unit.UnitTestProperties; |
3259 | |
3260 | public class VolumeTest extends PersistitUnitTestCase { |
3261 | |
3262 | === modified file 'src/test/java/com/persistit/WarmupTest.java' |
3263 | --- src/test/java/com/persistit/WarmupTest.java 2012-08-24 14:04:53 +0000 |
3264 | +++ src/test/java/com/persistit/WarmupTest.java 2012-08-24 14:04:53 +0000 |
3265 | @@ -16,52 +16,105 @@ |
3266 | package com.persistit; |
3267 | |
3268 | import static org.junit.Assert.assertEquals; |
3269 | +import static org.junit.Assert.assertTrue; |
3270 | |
3271 | import java.util.Properties; |
3272 | |
3273 | import org.junit.Test; |
3274 | |
3275 | -import com.persistit.unit.PersistitUnitTestCase; |
3276 | - |
3277 | public class WarmupTest extends PersistitUnitTestCase { |
3278 | |
3279 | + @Override |
3280 | + protected Properties getProperties(final boolean cleanup) { |
3281 | + final Properties p = super.getProperties(cleanup); |
3282 | + p.setProperty("bufferinventory", "true"); |
3283 | + p.setProperty("bufferpreload", "true"); |
3284 | + return p; |
3285 | + } |
3286 | + |
3287 | @Test |
3288 | public void testWarmup() throws Exception { |
3289 | Exchange ex = _persistit.getExchange("persistit", "WarmupTest", true); |
3290 | + final BufferPool pool = ex.getBufferPool(); |
3291 | for (int i = 1; i <= 1000; i++) { |
3292 | ex.getValue().put(RED_FOX); |
3293 | ex.clear().append(i).store(); |
3294 | } |
3295 | |
3296 | - // Assumption: only one buffer pool is created |
3297 | - int poolCount = 0; |
3298 | - String pathName = ""; |
3299 | final Buffer[] buff = new Buffer[100]; |
3300 | - for (final BufferPool p : _persistit.getBufferPoolHashMap().values()) { |
3301 | - poolCount = p.getBufferCount(); |
3302 | - pathName = p.toString(); |
3303 | - for (int i = 0; i < poolCount; ++i) { |
3304 | - buff[i] = p.getBufferCopy(i); |
3305 | - } |
3306 | - } |
3307 | - |
3308 | - final Properties properties = _persistit.getProperties(); |
3309 | - ex = null; |
3310 | - _persistit.close(); |
3311 | - |
3312 | - _persistit = new Persistit(); |
3313 | - _persistit.initialize(properties); |
3314 | - |
3315 | - int poolCount1 = 0; |
3316 | - for (final BufferPool p : _persistit.getBufferPoolHashMap().values()) { |
3317 | - poolCount1 = p.getBufferCount(); |
3318 | - for (int i = 0; i < poolCount1; ++i) { |
3319 | - final Buffer bufferCopy = p.getBufferCopy(i); |
3320 | - assertEquals(bufferCopy.getPageAddress(), buff[i].getPageAddress()); |
3321 | - assertEquals(bufferCopy.getPageType(), buff[i].getPageType()); |
3322 | - assertEquals(bufferCopy.getBufferSize(), buff[i].getBufferSize()); |
3323 | - } |
3324 | - } |
3325 | - assertEquals(poolCount, poolCount1); |
3326 | + for (int i = 0; i < pool.getBufferCount(); ++i) { |
3327 | + buff[i] = pool.getBufferCopy(i); |
3328 | + } |
3329 | + |
3330 | + final Configuration config = _persistit.getConfiguration(); |
3331 | + ex = null; |
3332 | + _persistit.close(); |
3333 | + |
3334 | + _persistit = new Persistit(); |
3335 | + _persistit.initialize(config); |
3336 | + |
3337 | + for (int i = 0; i < pool.getBufferCount(); ++i) { |
3338 | + final Buffer bufferCopy = pool.getBufferCopy(i); |
3339 | + assertEquals(bufferCopy.getPageAddress(), buff[i].getPageAddress()); |
3340 | + assertEquals(bufferCopy.getPageType(), buff[i].getPageType()); |
3341 | + assertEquals(bufferCopy.getBufferSize(), buff[i].getBufferSize()); |
3342 | + } |
3343 | + } |
3344 | + |
3345 | + @Test |
3346 | + public void readOrderIsSequential() throws Exception { |
3347 | + |
3348 | + Exchange ex = _persistit.getExchange("persistit", "WarmupTest", true); |
3349 | + BufferPool pool = ex.getBufferPool(); |
3350 | + |
3351 | + final int full = pool.getBufferCount() * (pool.getBufferSize() / RED_FOX.length()); |
3352 | + /* |
3353 | + * Overflow the buffer pool |
3354 | + */ |
3355 | + for (int i = 1; i <= full * 3; i++) { |
3356 | + ex.getValue().put(RED_FOX); |
3357 | + ex.clear().append(i).store(); |
3358 | + } |
3359 | + /* |
3360 | + * Pull some low-address pages in to scramble the pool |
3361 | + */ |
3362 | + for (int i = full * 2; i >= 0; i -= 1000) { |
3363 | + ex.clear().append(i).fetch(); |
3364 | + } |
3365 | + /* |
3366 | + * Verify that buffers in pool now have somewhat scrambled page |
3367 | + * addresses |
3368 | + */ |
3369 | + int breaks = 0; |
3370 | + long previous = -1; |
3371 | + |
3372 | + for (int i = 0; i < pool.getBufferCount(); i++) { |
3373 | + final Buffer b = pool.getBufferCopy(i); |
3374 | + assertTrue("Every buffer should be valid at this point", b.isValid()); |
3375 | + if (b.getPageAddress() < previous) { |
3376 | + breaks++; |
3377 | + } |
3378 | + previous = b.getPageAddress(); |
3379 | + } |
3380 | + |
3381 | + assertTrue("Buffer pool should have scrambled page address", breaks > 0); |
3382 | + |
3383 | + final Configuration config = _persistit.getConfiguration(); |
3384 | + ex = null; |
3385 | + pool = null; |
3386 | + _persistit.close(); |
3387 | + |
3388 | + _persistit = new Persistit(); |
3389 | + config.setBufferPreloadEnabled(false); |
3390 | + _persistit.initialize(config); |
3391 | + |
3392 | + final Volume volume = _persistit.getVolume("persistit"); |
3393 | + final MediatedFileChannel mfc = (MediatedFileChannel) volume.getStorage().getChannel(); |
3394 | + final TrackingFileChannel tfc = new TrackingFileChannel(); |
3395 | + mfc.injectChannelForTests(tfc); |
3396 | + pool = volume.getStructure().getPool(); |
3397 | + pool.preloadBufferInventory(); |
3398 | + assertTrue("Preload should have loaded pages from journal file", tfc.getReadPositionList().size() > 0); |
3399 | + tfc.assertSequential(true, true); |
3400 | } |
3401 | } |
3402 | |
3403 | === added file 'src/test/java/com/persistit/stress/PreloadMixtureTxn1.java' |
3404 | --- src/test/java/com/persistit/stress/PreloadMixtureTxn1.java 1970-01-01 00:00:00 +0000 |
3405 | +++ src/test/java/com/persistit/stress/PreloadMixtureTxn1.java 2012-08-24 14:04:53 +0000 |
3406 | @@ -0,0 +1,82 @@ |
3407 | +/** |
3408 | + * Copyright © 2012 Akiban Technologies, Inc. All rights reserved. |
3409 | + * |
3410 | + * This program and the accompanying materials are made available |
3411 | + * under the terms of the Eclipse Public License v1.0 which |
3412 | + * accompanies this distribution, and is available at |
3413 | + * http://www.eclipse.org/legal/epl-v10.html |
3414 | + * |
3415 | + * This program may also be available under different license terms. |
3416 | + * For more information, see www.akiban.com or contact licensing@akiban.com. |
3417 | + * |
3418 | + * Contributors: |
3419 | + * Akiban Technologies, Inc. |
3420 | + */ |
3421 | + |
3422 | +package com.persistit.stress; |
3423 | + |
3424 | +import com.persistit.Configuration; |
3425 | +import com.persistit.Persistit; |
3426 | +import com.persistit.Transaction.CommitPolicy; |
3427 | +import com.persistit.stress.unit.Stress1; |
3428 | +import com.persistit.stress.unit.Stress2txn; |
3429 | +import com.persistit.stress.unit.Stress3; |
3430 | +import com.persistit.stress.unit.Stress3txn; |
3431 | +import com.persistit.stress.unit.Stress5; |
3432 | +import com.persistit.stress.unit.Stress6; |
3433 | +import com.persistit.stress.unit.Stress8txn; |
3434 | + |
3435 | +public class PreloadMixtureTxn1 extends AbstractSuite { |
3436 | + private final static int CYCLES = 4; |
3437 | + |
3438 | + static String name() { |
3439 | + return PreloadMixtureTxn1.class.getSimpleName(); |
3440 | + } |
3441 | + |
3442 | + public static void main(final String[] args) throws Exception { |
3443 | + new PreloadMixtureTxn1(args).runTest(); |
3444 | + } |
3445 | + |
3446 | + public PreloadMixtureTxn1(final String[] args) { |
3447 | + super(name(), args); |
3448 | + setDuration(getDuration() / CYCLES); |
3449 | + } |
3450 | + |
3451 | + @Override |
3452 | + public void runTest() throws Exception { |
3453 | + |
3454 | + deleteFiles(substitute("$datapath$/persistit*")); |
3455 | + |
3456 | + for (int iteration = 0; iteration < CYCLES; iteration++) { |
3457 | + clear(); |
3458 | + add(new Stress1("repeat=10 count=25000")); |
3459 | + add(new Stress1("repeat=10 count=25000")); |
3460 | + add(new Stress2txn("repeat=10 count=2500 size=4000 seed=118")); |
3461 | + add(new Stress2txn("repeat=2 count=25000 seed=119")); |
3462 | + add(new Stress3("repeat=5 count=25000 seed=119")); |
3463 | + add(new Stress3txn("repeat=5 count=25000 seed=120")); |
3464 | + add(new Stress3txn("repeat=5 count=25000")); |
3465 | + add(new Stress5("repeat=5 count=25000")); |
3466 | + add(new Stress6("repeat=5 count=1000 size=250")); |
3467 | + add(new Stress6("repeat=10 count=1000 size=250")); |
3468 | + add(new Stress8txn("repeat=2 count=1000 size=1000 seed=1")); |
3469 | + add(new Stress8txn("repeat=2 count=1000 size=1000 seed=2")); |
3470 | + add(new Stress8txn("repeat=2 count=1000 size=1000 seed=3")); |
3471 | + add(new Stress8txn("repeat=2 count=1000 size=1000 seed=4")); |
3472 | + |
3473 | + final Configuration configuration = makeConfiguration(16384, "25000", CommitPolicy.SOFT); |
3474 | + configuration.setLogFile(configuration.getLogFile() + "_" + iteration); |
3475 | + configuration.setBufferInventoryEnabled(true); |
3476 | + configuration.setBufferPreloadEnabled(true); |
3477 | + |
3478 | + final Persistit persistit = new Persistit(); |
3479 | + persistit.initialize(configuration); |
3480 | + |
3481 | + try { |
3482 | + execute(persistit); |
3483 | + } finally { |
3484 | + persistit.close(); |
3485 | + } |
3486 | + } |
3487 | + } |
3488 | +} |
3489 | |
3490 | === modified file 'src/test/java/com/persistit/stress/unit/CommitBench.java' |
3491 | --- src/test/java/com/persistit/stress/unit/CommitBench.java 2012-08-24 14:04:53 +0000 |
3492 | +++ src/test/java/com/persistit/stress/unit/CommitBench.java 2012-08-24 14:04:53 +0000 |
3493 | @@ -15,15 +15,19 @@ |
3494 | |
3495 | package com.persistit.stress.unit; |
3496 | |
3497 | +import java.io.File; |
3498 | +import java.io.IOException; |
3499 | +import java.io.RandomAccessFile; |
3500 | import java.util.Properties; |
3501 | import java.util.Random; |
3502 | import java.util.concurrent.atomic.AtomicInteger; |
3503 | +import java.util.regex.Pattern; |
3504 | |
3505 | import com.persistit.Exchange; |
3506 | +import com.persistit.PersistitUnitTestCase; |
3507 | import com.persistit.Transaction; |
3508 | import com.persistit.Transaction.CommitPolicy; |
3509 | import com.persistit.exception.RollbackException; |
3510 | -import com.persistit.unit.PersistitUnitTestCase; |
3511 | import com.persistit.unit.UnitTestProperties; |
3512 | import com.persistit.util.ArgParser; |
3513 | |
3514 | @@ -38,6 +42,9 @@ |
3515 | * |
3516 | */ |
3517 | public class CommitBench extends PersistitUnitTestCase { |
3518 | + |
3519 | + final static Pattern PATH_PATTERN = Pattern.compile("(.+)\\.(\\d{12})"); |
3520 | + |
3521 | /* |
3522 | * Want about 1000 pages worth of data (no evictions). Each record is about |
3523 | * 50 bytes, so 1000 * 16384 * 60% / 50 ~= 200000. |
3524 | @@ -47,19 +54,56 @@ |
3525 | private final int RECORDS_PER_TXN = 10; |
3526 | private final String[] ARG_TEMPLATE = new String[] { "threads|int:1:1:1000|Number of threads", |
3527 | "duration|int:10:10:86400|Duration of test in seconds", |
3528 | - "policy|String:HARD|Commit policy: SOFT, HARD or GROUP", }; |
3529 | + "policy|String:HARD|Commit policy: SOFT, HARD or GROUP", "datapath|String|Datapath property", |
3530 | + "_flag|P|Reuse journal file" }; |
3531 | |
3532 | volatile long stopTime; |
3533 | + final ArgParser ap; |
3534 | AtomicInteger commitCount = new AtomicInteger(); |
3535 | AtomicInteger rollbackCount = new AtomicInteger(); |
3536 | |
3537 | + CommitBench(final String[] args) { |
3538 | + ap = new ArgParser("CommitBench", args, ARG_TEMPLATE).strict(); |
3539 | + } |
3540 | + |
3541 | @Override |
3542 | protected Properties getProperties(final boolean cleanup) { |
3543 | - return UnitTestProperties.getBiggerProperties(cleanup); |
3544 | + final Properties p = UnitTestProperties.getBiggerProperties(false); |
3545 | + if (ap.isSpecified("datapath")) { |
3546 | + p.setProperty("datapath", ap.getStringValue("datapath")); |
3547 | + } |
3548 | + /* |
3549 | + * Custom data directory cleanup - leaving the journal file behind if |
3550 | + */ |
3551 | + final String path = p.getProperty("datapath"); |
3552 | + final File dir = new File(path); |
3553 | + assert dir.isDirectory() : "Data path does not specify a directory: " + path; |
3554 | + final File[] files = dir.listFiles(); |
3555 | + for (final File file : files) { |
3556 | + if (ap.isFlag('P') && PATH_PATTERN.matcher(file.getName()).matches()) { |
3557 | + try { |
3558 | + /* |
3559 | + * Damage the file so that there's no keystone checkpoint |
3560 | + */ |
3561 | + RandomAccessFile raf = new RandomAccessFile(file, "rws"); |
3562 | + raf.seek(0); |
3563 | + raf.write(new byte[256]); |
3564 | + raf.close(); |
3565 | + } catch (IOException e) { |
3566 | + throw new RuntimeException(e); |
3567 | + } |
3568 | + } else { |
3569 | + if (file.isDirectory()) { |
3570 | + UnitTestProperties.cleanUpDirectory(file); |
3571 | + } else { |
3572 | + file.delete(); |
3573 | + } |
3574 | + } |
3575 | + } |
3576 | + return p; |
3577 | } |
3578 | |
3579 | - public void bench(final String[] args) throws Exception { |
3580 | - final ArgParser ap = new ArgParser("CommitBench", args, ARG_TEMPLATE).strict(); |
3581 | + public void bench() throws Exception { |
3582 | final int threadCount = ap.getIntValue("threads"); |
3583 | final int duration = ap.getIntValue("duration"); |
3584 | final String policy = ap.getStringValue("policy"); |
3585 | @@ -138,10 +182,10 @@ |
3586 | } |
3587 | |
3588 | public static void main(final String[] args) throws Exception { |
3589 | - final CommitBench bench = new CommitBench(); |
3590 | + final CommitBench bench = new CommitBench(args); |
3591 | try { |
3592 | bench.setUp(); |
3593 | - bench.bench(args); |
3594 | + bench.bench(); |
3595 | } finally { |
3596 | bench.tearDown(); |
3597 | } |
3598 | |
3599 | === modified file 'src/test/java/com/persistit/stress/unit/TestSamePageOptimization.java' |
3600 | --- src/test/java/com/persistit/stress/unit/TestSamePageOptimization.java 2012-08-24 14:04:53 +0000 |
3601 | +++ src/test/java/com/persistit/stress/unit/TestSamePageOptimization.java 2012-08-24 14:04:53 +0000 |
3602 | @@ -19,8 +19,8 @@ |
3603 | |
3604 | import com.persistit.Exchange; |
3605 | import com.persistit.Key; |
3606 | +import com.persistit.PersistitUnitTestCase; |
3607 | import com.persistit.exception.PersistitException; |
3608 | -import com.persistit.unit.PersistitUnitTestCase; |
3609 | |
3610 | public class TestSamePageOptimization extends PersistitUnitTestCase { |
3611 | final int KEYS = 1000000; |
3612 | |
3613 | === modified file 'src/test/java/com/persistit/unit/BufferPoolMemConfigurationTest.java' |
3614 | --- src/test/java/com/persistit/unit/BufferPoolMemConfigurationTest.java 2012-08-24 14:04:53 +0000 |
3615 | +++ src/test/java/com/persistit/unit/BufferPoolMemConfigurationTest.java 2012-08-24 14:04:53 +0000 |
3616 | @@ -24,6 +24,7 @@ |
3617 | import org.junit.Test; |
3618 | |
3619 | import com.persistit.Management.BufferPoolInfo; |
3620 | +import com.persistit.PersistitUnitTestCase; |
3621 | |
3622 | public class BufferPoolMemConfigurationTest extends PersistitUnitTestCase { |
3623 | |
3624 | |
3625 | === modified file 'src/test/java/com/persistit/unit/DefaultCoderManagerTest.java' |
3626 | --- src/test/java/com/persistit/unit/DefaultCoderManagerTest.java 2012-08-02 04:45:28 +0000 |
3627 | +++ src/test/java/com/persistit/unit/DefaultCoderManagerTest.java 2012-08-24 14:04:53 +0000 |
3628 | @@ -25,6 +25,7 @@ |
3629 | import org.junit.Test; |
3630 | |
3631 | import com.persistit.DefaultCoderManager; |
3632 | +import com.persistit.PersistitUnitTestCase; |
3633 | import com.persistit.exception.PersistitException; |
3634 | |
3635 | public class DefaultCoderManagerTest extends PersistitUnitTestCase implements Serializable { |
3636 | |
3637 | === modified file 'src/test/java/com/persistit/unit/ExchangeTest.java' |
3638 | --- src/test/java/com/persistit/unit/ExchangeTest.java 2012-08-24 14:04:53 +0000 |
3639 | +++ src/test/java/com/persistit/unit/ExchangeTest.java 2012-08-24 14:04:53 +0000 |
3640 | @@ -28,6 +28,7 @@ |
3641 | import com.persistit.Exchange; |
3642 | import com.persistit.Key; |
3643 | import com.persistit.KeyFilter; |
3644 | +import com.persistit.PersistitUnitTestCase; |
3645 | import com.persistit.Transaction; |
3646 | import com.persistit.Volume; |
3647 | import com.persistit.exception.ConversionException; |
3648 | |
3649 | === modified file 'src/test/java/com/persistit/unit/FileLockTest.java' |
3650 | --- src/test/java/com/persistit/unit/FileLockTest.java 2012-08-24 14:04:53 +0000 |
3651 | +++ src/test/java/com/persistit/unit/FileLockTest.java 2012-08-24 14:04:53 +0000 |
3652 | @@ -22,6 +22,7 @@ |
3653 | import org.junit.Test; |
3654 | |
3655 | import com.persistit.Persistit; |
3656 | +import com.persistit.PersistitUnitTestCase; |
3657 | import com.persistit.exception.PersistitException; |
3658 | |
3659 | /** |
3660 | |
3661 | === modified file 'src/test/java/com/persistit/unit/JoinTest1.java' |
3662 | --- src/test/java/com/persistit/unit/JoinTest1.java 2012-08-24 14:04:53 +0000 |
3663 | +++ src/test/java/com/persistit/unit/JoinTest1.java 2012-08-24 14:04:53 +0000 |
3664 | @@ -21,6 +21,7 @@ |
3665 | |
3666 | import com.persistit.Exchange; |
3667 | import com.persistit.Key; |
3668 | +import com.persistit.PersistitUnitTestCase; |
3669 | import com.persistit.Value; |
3670 | import com.persistit.exception.PersistitException; |
3671 | |
3672 | |
3673 | === modified file 'src/test/java/com/persistit/unit/KeyCoderTest1.java' |
3674 | --- src/test/java/com/persistit/unit/KeyCoderTest1.java 2012-08-24 14:04:53 +0000 |
3675 | +++ src/test/java/com/persistit/unit/KeyCoderTest1.java 2012-08-24 14:04:53 +0000 |
3676 | @@ -27,6 +27,7 @@ |
3677 | import org.junit.Test; |
3678 | |
3679 | import com.persistit.Key; |
3680 | +import com.persistit.PersistitUnitTestCase; |
3681 | import com.persistit.TestShim; |
3682 | import com.persistit.encoding.CoderContext; |
3683 | import com.persistit.encoding.KeyCoder; |
3684 | |
3685 | === modified file 'src/test/java/com/persistit/unit/KeyFilterTest1.java' |
3686 | --- src/test/java/com/persistit/unit/KeyFilterTest1.java 2012-08-02 04:45:28 +0000 |
3687 | +++ src/test/java/com/persistit/unit/KeyFilterTest1.java 2012-08-24 14:04:53 +0000 |
3688 | @@ -28,6 +28,7 @@ |
3689 | import com.persistit.KeyFilter; |
3690 | import com.persistit.KeyState; |
3691 | import com.persistit.Persistit; |
3692 | +import com.persistit.PersistitUnitTestCase; |
3693 | import com.persistit.exception.PersistitException; |
3694 | |
3695 | /** |
3696 | |
3697 | === modified file 'src/test/java/com/persistit/unit/KeyFilterTest2.java' |
3698 | --- src/test/java/com/persistit/unit/KeyFilterTest2.java 2012-08-24 14:04:53 +0000 |
3699 | +++ src/test/java/com/persistit/unit/KeyFilterTest2.java 2012-08-24 14:04:53 +0000 |
3700 | @@ -23,6 +23,7 @@ |
3701 | import com.persistit.Exchange; |
3702 | import com.persistit.Key; |
3703 | import com.persistit.KeyFilter; |
3704 | +import com.persistit.PersistitUnitTestCase; |
3705 | import com.persistit.exception.PersistitException; |
3706 | |
3707 | /** |
3708 | |
3709 | === modified file 'src/test/java/com/persistit/unit/KeyParserTest1.java' |
3710 | --- src/test/java/com/persistit/unit/KeyParserTest1.java 2012-08-02 04:45:28 +0000 |
3711 | +++ src/test/java/com/persistit/unit/KeyParserTest1.java 2012-08-24 14:04:53 +0000 |
3712 | @@ -23,6 +23,7 @@ |
3713 | import com.persistit.Key; |
3714 | import com.persistit.KeyParser; |
3715 | import com.persistit.Persistit; |
3716 | +import com.persistit.PersistitUnitTestCase; |
3717 | |
3718 | /** |
3719 | * @version 1.0 |
3720 | |
3721 | === modified file 'src/test/java/com/persistit/unit/KeyParserTest2.java' |
3722 | --- src/test/java/com/persistit/unit/KeyParserTest2.java 2012-08-02 04:45:28 +0000 |
3723 | +++ src/test/java/com/persistit/unit/KeyParserTest2.java 2012-08-24 14:04:53 +0000 |
3724 | @@ -23,6 +23,7 @@ |
3725 | import com.persistit.Key; |
3726 | import com.persistit.KeyFilter; |
3727 | import com.persistit.KeyParser; |
3728 | +import com.persistit.PersistitUnitTestCase; |
3729 | |
3730 | public class KeyParserTest2 extends PersistitUnitTestCase { |
3731 | |
3732 | |
3733 | === modified file 'src/test/java/com/persistit/unit/KeyTest1.java' |
3734 | --- src/test/java/com/persistit/unit/KeyTest1.java 2012-08-24 14:04:53 +0000 |
3735 | +++ src/test/java/com/persistit/unit/KeyTest1.java 2012-08-24 14:04:53 +0000 |
3736 | @@ -30,6 +30,7 @@ |
3737 | import com.persistit.Buffer; |
3738 | import com.persistit.Key; |
3739 | import com.persistit.KeyState; |
3740 | +import com.persistit.PersistitUnitTestCase; |
3741 | import com.persistit.TestShim; |
3742 | import com.persistit.exception.InvalidKeyException; |
3743 | import com.persistit.exception.MissingKeySegmentException; |
3744 | |
3745 | === modified file 'src/test/java/com/persistit/unit/Log4JLoggerTest.java' |
3746 | --- src/test/java/com/persistit/unit/Log4JLoggerTest.java 2012-08-24 14:04:53 +0000 |
3747 | +++ src/test/java/com/persistit/unit/Log4JLoggerTest.java 2012-08-24 14:04:53 +0000 |
3748 | @@ -20,6 +20,7 @@ |
3749 | import org.junit.Test; |
3750 | |
3751 | import com.persistit.Persistit; |
3752 | +import com.persistit.PersistitUnitTestCase; |
3753 | import com.persistit.exception.PersistitException; |
3754 | import com.persistit.logging.Log4JAdapter; |
3755 | |
3756 | |
3757 | === modified file 'src/test/java/com/persistit/unit/LongRecordTest1.java' |
3758 | --- src/test/java/com/persistit/unit/LongRecordTest1.java 2012-08-24 14:04:53 +0000 |
3759 | +++ src/test/java/com/persistit/unit/LongRecordTest1.java 2012-08-24 14:04:53 +0000 |
3760 | @@ -22,6 +22,7 @@ |
3761 | |
3762 | import com.persistit.Exchange; |
3763 | import com.persistit.Key; |
3764 | +import com.persistit.PersistitUnitTestCase; |
3765 | import com.persistit.exception.PersistitException; |
3766 | import com.persistit.util.Debug; |
3767 | import com.persistit.util.Util; |
3768 | |
3769 | === modified file 'src/test/java/com/persistit/unit/LotsaSmallKeys.java' |
3770 | --- src/test/java/com/persistit/unit/LotsaSmallKeys.java 2012-08-24 14:04:53 +0000 |
3771 | +++ src/test/java/com/persistit/unit/LotsaSmallKeys.java 2012-08-24 14:04:53 +0000 |
3772 | @@ -20,6 +20,7 @@ |
3773 | import org.junit.Test; |
3774 | |
3775 | import com.persistit.Exchange; |
3776 | +import com.persistit.PersistitUnitTestCase; |
3777 | import com.persistit.exception.PersistitException; |
3778 | import com.persistit.policy.SplitPolicy; |
3779 | |
3780 | |
3781 | === modified file 'src/test/java/com/persistit/unit/PersistitMapTest.java' |
3782 | --- src/test/java/com/persistit/unit/PersistitMapTest.java 2012-08-24 14:04:53 +0000 |
3783 | +++ src/test/java/com/persistit/unit/PersistitMapTest.java 2012-08-24 14:04:53 +0000 |
3784 | @@ -34,6 +34,7 @@ |
3785 | import com.persistit.Exchange; |
3786 | import com.persistit.KeyFilter; |
3787 | import com.persistit.PersistitMap; |
3788 | +import com.persistit.PersistitUnitTestCase; |
3789 | import com.persistit.exception.PersistitException; |
3790 | import com.persistit.util.Util; |
3791 | |
3792 | |
3793 | === removed file 'src/test/java/com/persistit/unit/PersistitUnitTestCase.java' |
3794 | --- src/test/java/com/persistit/unit/PersistitUnitTestCase.java 2012-08-24 14:04:53 +0000 |
3795 | +++ src/test/java/com/persistit/unit/PersistitUnitTestCase.java 1970-01-01 00:00:00 +0000 |
3796 | @@ -1,132 +0,0 @@ |
3797 | -/** |
3798 | - * Copyright © 2011-2012 Akiban Technologies, Inc. All rights reserved. |
3799 | - * |
3800 | - * This program and the accompanying materials are made available |
3801 | - * under the terms of the Eclipse Public License v1.0 which |
3802 | - * accompanies this distribution, and is available at |
3803 | - * http://www.eclipse.org/legal/epl-v10.html |
3804 | - * |
3805 | - * This program may also be available under different license terms. |
3806 | - * For more information, see www.akiban.com or contact licensing@akiban.com. |
3807 | - * |
3808 | - * Contributors: |
3809 | - * Akiban Technologies, Inc. |
3810 | - */ |
3811 | - |
3812 | -package com.persistit.unit; |
3813 | - |
3814 | -import java.lang.ref.WeakReference; |
3815 | -import java.util.Map; |
3816 | -import java.util.Properties; |
3817 | - |
3818 | -import org.junit.After; |
3819 | -import org.junit.Before; |
3820 | - |
3821 | -import com.persistit.Persistit; |
3822 | -import com.persistit.exception.PersistitException; |
3823 | - |
3824 | -public abstract class PersistitUnitTestCase { |
3825 | - |
3826 | - private final static long TEN_SECONDS = 10L * 1000L * 1000L * 1000L; |
3827 | - |
3828 | - protected final static String RED_FOX = "The quick red fox jumped over the lazy brown dog."; |
3829 | - |
3830 | - protected static String createString(final int exactLength) { |
3831 | - final StringBuilder sb = new StringBuilder(exactLength); |
3832 | - // Simple 0..9a..z string |
3833 | - for (int i = 0; i < 36; ++i) { |
3834 | - sb.append(Character.forDigit(i, 36)); |
3835 | - } |
3836 | - final String numAndLetters = sb.toString(); |
3837 | - while (sb.length() < exactLength) { |
3838 | - sb.append(numAndLetters); |
3839 | - } |
3840 | - return sb.toString().substring(0, exactLength); |
3841 | - } |
3842 | - |
3843 | - protected Persistit _persistit = new Persistit(); |
3844 | - |
3845 | - protected Properties getProperties(final boolean cleanup) { |
3846 | - return UnitTestProperties.getProperties(cleanup); |
3847 | - } |
3848 | - |
3849 | - @Before |
3850 | - public void setUp() throws Exception { |
3851 | - checkNoPersistitThreads(); |
3852 | - |
3853 | - _persistit.initialize(getProperties(true)); |
3854 | - } |
3855 | - |
3856 | - @After |
3857 | - public void tearDown() throws Exception { |
3858 | - final WeakReference<Persistit> ref = new WeakReference<Persistit>(_persistit); |
3859 | - _persistit.close(false); |
3860 | - _persistit = null; |
3861 | - |
3862 | - if (!doesRefBecomeNull(ref)) { |
3863 | - System.out.println("Persistit has a leftover strong reference"); |
3864 | - } |
3865 | - checkNoPersistitThreads(); |
3866 | - } |
3867 | - |
3868 | - public void runAllTests() throws Exception { |
3869 | - |
3870 | - } |
3871 | - |
3872 | - public void setPersistit(final Persistit persistit) { |
3873 | - _persistit = persistit; |
3874 | - } |
3875 | - |
3876 | - protected void initAndRunTest() throws Exception { |
3877 | - setUp(); |
3878 | - try { |
3879 | - runAllTests(); |
3880 | - } catch (final Throwable t) { |
3881 | - t.printStackTrace(); |
3882 | - } finally { |
3883 | - tearDown(); |
3884 | - } |
3885 | - } |
3886 | - |
3887 | - private final static String[] PERSISTIT_THREAD_NAMES = { "CHECKPOINT_WRITER", "JOURNAL_COPIER", "JOURNAL_FLUSHER", |
3888 | - "PAGE_WRITER", "TXN_UPDATE" }; |
3889 | - |
3890 | - protected boolean checkNoPersistitThreads() { |
3891 | - boolean alive = false; |
3892 | - final Map<Thread, StackTraceElement[]> map = Thread.getAllStackTraces(); |
3893 | - for (final Thread t : map.keySet()) { |
3894 | - final String name = t.getName(); |
3895 | - for (final String p : PERSISTIT_THREAD_NAMES) { |
3896 | - if (name.contains(p)) { |
3897 | - alive = true; |
3898 | - System.err.println("Thread " + t + " is still alive"); |
3899 | - } |
3900 | - } |
3901 | - } |
3902 | - return alive; |
3903 | - } |
3904 | - |
3905 | - protected void safeCrashAndRestoreProperties() throws PersistitException { |
3906 | - final Properties properties = _persistit.getProperties(); |
3907 | - _persistit.flush(); |
3908 | - _persistit.crash(); |
3909 | - _persistit = new Persistit(); |
3910 | - _persistit.initialize(properties); |
3911 | - } |
3912 | - |
3913 | - protected void crashWithoutFlushAndRestoreProperties() throws PersistitException { |
3914 | - final Properties properties = _persistit.getProperties(); |
3915 | - _persistit.crash(); |
3916 | - _persistit = new Persistit(); |
3917 | - _persistit.initialize(properties); |
3918 | - } |
3919 | - |
3920 | - public static boolean doesRefBecomeNull(final WeakReference<?> ref) throws InterruptedException { |
3921 | - final long expires = System.nanoTime() + TEN_SECONDS; |
3922 | - while (ref.get() != null && System.nanoTime() < expires) { |
3923 | - System.gc(); |
3924 | - Thread.sleep(10); |
3925 | - } |
3926 | - return ref.get() == null; |
3927 | - } |
3928 | -} |
3929 | |
3930 | === modified file 'src/test/java/com/persistit/unit/SaveLoadTest1.java' |
3931 | --- src/test/java/com/persistit/unit/SaveLoadTest1.java 2012-08-02 04:45:28 +0000 |
3932 | +++ src/test/java/com/persistit/unit/SaveLoadTest1.java 2012-08-24 14:04:53 +0000 |
3933 | @@ -24,6 +24,7 @@ |
3934 | import org.junit.Test; |
3935 | |
3936 | import com.persistit.PersistitMap; |
3937 | +import com.persistit.PersistitUnitTestCase; |
3938 | import com.persistit.StreamLoader; |
3939 | import com.persistit.StreamSaver; |
3940 | import com.persistit.exception.PersistitException; |
3941 | |
3942 | === modified file 'src/test/java/com/persistit/unit/SimpleTest1.java' |
3943 | --- src/test/java/com/persistit/unit/SimpleTest1.java 2012-08-24 14:04:53 +0000 |
3944 | +++ src/test/java/com/persistit/unit/SimpleTest1.java 2012-08-24 14:04:53 +0000 |
3945 | @@ -25,6 +25,7 @@ |
3946 | |
3947 | import com.persistit.Exchange; |
3948 | import com.persistit.Key; |
3949 | +import com.persistit.PersistitUnitTestCase; |
3950 | import com.persistit.Value; |
3951 | import com.persistit.exception.PersistitException; |
3952 | |
3953 | |
3954 | === modified file 'src/test/java/com/persistit/unit/TemporaryVolumeTest1.java' |
3955 | --- src/test/java/com/persistit/unit/TemporaryVolumeTest1.java 2012-08-24 14:04:53 +0000 |
3956 | +++ src/test/java/com/persistit/unit/TemporaryVolumeTest1.java 2012-08-24 14:04:53 +0000 |
3957 | @@ -28,6 +28,7 @@ |
3958 | import com.persistit.Exchange; |
3959 | import com.persistit.Key; |
3960 | import com.persistit.Management; |
3961 | +import com.persistit.PersistitUnitTestCase; |
3962 | import com.persistit.Value; |
3963 | import com.persistit.Volume; |
3964 | import com.persistit.exception.PersistitException; |
3965 | |
3966 | === modified file 'src/test/java/com/persistit/unit/TransactionTest1.java' |
3967 | --- src/test/java/com/persistit/unit/TransactionTest1.java 2012-08-24 14:04:53 +0000 |
3968 | +++ src/test/java/com/persistit/unit/TransactionTest1.java 2012-08-24 14:04:53 +0000 |
3969 | @@ -28,6 +28,7 @@ |
3970 | import com.persistit.Exchange; |
3971 | import com.persistit.Key; |
3972 | import com.persistit.KeyFilter; |
3973 | +import com.persistit.PersistitUnitTestCase; |
3974 | import com.persistit.Transaction; |
3975 | import com.persistit.Value; |
3976 | import com.persistit.exception.PersistitException; |
3977 | |
3978 | === modified file 'src/test/java/com/persistit/unit/TransactionTest3.java' |
3979 | --- src/test/java/com/persistit/unit/TransactionTest3.java 2012-08-24 14:04:53 +0000 |
3980 | +++ src/test/java/com/persistit/unit/TransactionTest3.java 2012-08-24 14:04:53 +0000 |
3981 | @@ -27,6 +27,7 @@ |
3982 | |
3983 | import com.persistit.Exchange; |
3984 | import com.persistit.Key; |
3985 | +import com.persistit.PersistitUnitTestCase; |
3986 | import com.persistit.Transaction; |
3987 | import com.persistit.exception.PersistitException; |
3988 | |
3989 | |
3990 | === modified file 'src/test/java/com/persistit/unit/TreeTest1.java' |
3991 | --- src/test/java/com/persistit/unit/TreeTest1.java 2012-08-02 04:45:28 +0000 |
3992 | +++ src/test/java/com/persistit/unit/TreeTest1.java 2012-08-24 14:04:53 +0000 |
3993 | @@ -23,6 +23,7 @@ |
3994 | |
3995 | import com.persistit.Exchange; |
3996 | import com.persistit.Key; |
3997 | +import com.persistit.PersistitUnitTestCase; |
3998 | |
3999 | public class TreeTest1 extends PersistitUnitTestCase { |
4000 | |
4001 | |
4002 | === modified file 'src/test/java/com/persistit/unit/UnitTestProperties.java' |
4003 | --- src/test/java/com/persistit/unit/UnitTestProperties.java 2012-08-07 21:11:28 +0000 |
4004 | +++ src/test/java/com/persistit/unit/UnitTestProperties.java 2012-08-24 14:04:53 +0000 |
4005 | @@ -43,7 +43,6 @@ |
4006 | p.setProperty("tmpvoldir", "${datapath}"); |
4007 | p.setProperty("rmiport", System.getProperty("rmiport", "8081")); |
4008 | p.setProperty("jmx", "true"); |
4009 | - p.setProperty("bufferinventory", "/tmp/persistit_test_data"); |
4010 | return p; |
4011 | } |
4012 | |
4013 | |
4014 | === modified file 'src/test/java/com/persistit/unit/ValueCoderTest1.java' |
4015 | --- src/test/java/com/persistit/unit/ValueCoderTest1.java 2012-08-24 14:04:53 +0000 |
4016 | +++ src/test/java/com/persistit/unit/ValueCoderTest1.java 2012-08-24 14:04:53 +0000 |
4017 | @@ -28,6 +28,7 @@ |
4018 | import org.junit.Test; |
4019 | |
4020 | import com.persistit.Exchange; |
4021 | +import com.persistit.PersistitUnitTestCase; |
4022 | import com.persistit.Value; |
4023 | import com.persistit.encoding.CoderContext; |
4024 | import com.persistit.encoding.ValueCoder; |
4025 | |
4026 | === modified file 'src/test/java/com/persistit/unit/ValueCoderTest2.java' |
4027 | --- src/test/java/com/persistit/unit/ValueCoderTest2.java 2012-08-02 04:45:28 +0000 |
4028 | +++ src/test/java/com/persistit/unit/ValueCoderTest2.java 2012-08-24 14:04:53 +0000 |
4029 | @@ -25,6 +25,7 @@ |
4030 | |
4031 | import com.persistit.DefaultObjectCoder; |
4032 | import com.persistit.Exchange; |
4033 | +import com.persistit.PersistitUnitTestCase; |
4034 | import com.persistit.exception.PersistitException; |
4035 | |
4036 | public class ValueCoderTest2 extends PersistitUnitTestCase { |
4037 | |
4038 | === modified file 'src/test/java/com/persistit/unit/ValueTest1.java' |
4039 | --- src/test/java/com/persistit/unit/ValueTest1.java 2012-08-24 14:04:53 +0000 |
4040 | +++ src/test/java/com/persistit/unit/ValueTest1.java 2012-08-24 14:04:53 +0000 |
4041 | @@ -28,6 +28,7 @@ |
4042 | |
4043 | import org.junit.Test; |
4044 | |
4045 | +import com.persistit.PersistitUnitTestCase; |
4046 | import com.persistit.Value; |
4047 | import com.persistit.util.Util; |
4048 | |
4049 | |
4050 | === modified file 'src/test/java/com/persistit/unit/ValueTest2.java' |
4051 | --- src/test/java/com/persistit/unit/ValueTest2.java 2012-08-24 14:04:53 +0000 |
4052 | +++ src/test/java/com/persistit/unit/ValueTest2.java 2012-08-24 14:04:53 +0000 |
4053 | @@ -30,6 +30,7 @@ |
4054 | import org.junit.Test; |
4055 | |
4056 | import com.persistit.Exchange; |
4057 | +import com.persistit.PersistitUnitTestCase; |
4058 | import com.persistit.Value; |
4059 | import com.persistit.encoding.CollectionValueCoder; |
4060 | import com.persistit.exception.PersistitException; |
4061 | |
4062 | === modified file 'src/test/java/com/persistit/unit/ValueTest3.java' |
4063 | --- src/test/java/com/persistit/unit/ValueTest3.java 2012-08-24 14:04:53 +0000 |
4064 | +++ src/test/java/com/persistit/unit/ValueTest3.java 2012-08-24 14:04:53 +0000 |
4065 | @@ -34,6 +34,7 @@ |
4066 | import com.persistit.DefaultValueCoder; |
4067 | import com.persistit.Exchange; |
4068 | import com.persistit.Persistit; |
4069 | +import com.persistit.PersistitUnitTestCase; |
4070 | import com.persistit.Value; |
4071 | import com.persistit.encoding.CoderContext; |
4072 | import com.persistit.encoding.CoderManager; |
4073 | |
4074 | === modified file 'src/test/java/com/persistit/unit/ValueTest4.java' |
4075 | --- src/test/java/com/persistit/unit/ValueTest4.java 2012-08-24 14:04:53 +0000 |
4076 | +++ src/test/java/com/persistit/unit/ValueTest4.java 2012-08-24 14:04:53 +0000 |
4077 | @@ -35,6 +35,7 @@ |
4078 | import com.persistit.DefaultValueCoder; |
4079 | import com.persistit.Exchange; |
4080 | import com.persistit.Persistit; |
4081 | +import com.persistit.PersistitUnitTestCase; |
4082 | import com.persistit.Value; |
4083 | import com.persistit.encoding.CoderContext; |
4084 | import com.persistit.encoding.CoderManager; |
4085 | |
4086 | === modified file 'src/test/java/com/persistit/unit/ValueTest5.java' |
4087 | --- src/test/java/com/persistit/unit/ValueTest5.java 2012-08-24 14:04:53 +0000 |
4088 | +++ src/test/java/com/persistit/unit/ValueTest5.java 2012-08-24 14:04:53 +0000 |
4089 | @@ -22,6 +22,7 @@ |
4090 | import org.junit.Test; |
4091 | |
4092 | import com.persistit.Exchange; |
4093 | +import com.persistit.PersistitUnitTestCase; |
4094 | import com.persistit.encoding.CoderManager; |
4095 | import com.persistit.exception.PersistitException; |
4096 |