PageRenderTime 85ms CodeModel.GetById 17ms RepoModel.GetById 1ms app.codeStats 0ms

/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java

https://github.com/stephenc/cassandra
Java | 269 lines | 214 code | 28 blank | 27 comment | 38 complexity | b0ebfac3e9234ff090cc124c29df4f39 MD5 | raw file
  1. package org.apache.cassandra.db.commitlog;
  2. import java.io.DataInputStream;
  3. import java.io.EOFException;
  4. import java.io.File;
  5. import java.io.IOException;
  6. import java.util.ArrayList;
  7. import java.util.HashMap;
  8. import java.util.List;
  9. import java.util.Map;
  10. import java.util.Set;
  11. import java.util.concurrent.Future;
  12. import java.util.concurrent.atomic.AtomicInteger;
  13. import java.util.zip.Checksum;
  14. import org.apache.cassandra.concurrent.Stage;
  15. import org.apache.cassandra.concurrent.StageManager;
  16. import org.apache.cassandra.config.Schema;
  17. import org.apache.cassandra.db.ColumnFamily;
  18. import org.apache.cassandra.db.ColumnFamilyStore;
  19. import org.apache.cassandra.db.RowMutation;
  20. import org.apache.cassandra.db.Table;
  21. import org.apache.cassandra.db.UnknownColumnFamilyException;
  22. import org.apache.cassandra.io.IColumnSerializer;
  23. import org.apache.cassandra.io.util.FastByteArrayInputStream;
  24. import org.apache.cassandra.io.util.FileUtils;
  25. import org.apache.cassandra.io.util.RandomAccessReader;
  26. import org.apache.cassandra.net.MessagingService;
  27. import org.apache.cassandra.utils.ByteBufferUtil;
  28. import org.apache.cassandra.utils.FBUtilities;
  29. import org.apache.cassandra.utils.PureJavaCrc32;
  30. import org.apache.cassandra.utils.WrappedRunnable;
  31. import org.apache.commons.lang.StringUtils;
  32. import org.cliffc.high_scale_lib.NonBlockingHashSet;
  33. import org.slf4j.Logger;
  34. import org.slf4j.LoggerFactory;
  35. import com.google.common.collect.Ordering;
  36. public class CommitLogReplayer
  37. {
  38. private static final Logger logger = LoggerFactory.getLogger(CommitLogReplayer.class);
  39. private static final int MAX_OUTSTANDING_REPLAY_COUNT = 1024;
  40. private final Set<Table> tablesRecovered;
  41. private final List<Future<?>> futures;
  42. private final Map<Integer, AtomicInteger> invalidMutations;
  43. private final AtomicInteger replayedCount;
  44. private final Map<Integer, ReplayPosition> cfPositions;
  45. private final ReplayPosition globalPosition;
  46. private final Checksum checksum;
  47. private byte[] buffer;
  48. public CommitLogReplayer()
  49. {
  50. this.tablesRecovered = new NonBlockingHashSet<Table>();
  51. this.futures = new ArrayList<Future<?>>();
  52. this.buffer = new byte[4096];
  53. this.invalidMutations = new HashMap<Integer, AtomicInteger>();
  54. // count the number of replayed mutation. We don't really care about atomicity, but we need it to be a reference.
  55. this.replayedCount = new AtomicInteger();
  56. // compute per-CF and global replay positions
  57. this.cfPositions = new HashMap<Integer, ReplayPosition>();
  58. for (ColumnFamilyStore cfs : ColumnFamilyStore.all())
  59. {
  60. // it's important to call RP.gRP per-cf, before aggregating all the positions w/ the Ordering.min call
  61. // below: gRP will return NONE if there are no flushed sstables, which is important to have in the
  62. // list (otherwise we'll just start replay from the first flush position that we do have, which is not correct).
  63. ReplayPosition rp = ReplayPosition.getReplayPosition(cfs.getSSTables());
  64. cfPositions.put(cfs.metadata.cfId, rp);
  65. }
  66. this.globalPosition = Ordering.from(ReplayPosition.comparator).min(cfPositions.values());
  67. this.checksum = new PureJavaCrc32();
  68. }
  69. public void recover(File[] clogs) throws IOException
  70. {
  71. for (final File file : clogs)
  72. recover(file);
  73. }
  74. public int blockForWrites() throws IOException
  75. {
  76. for (Map.Entry<Integer, AtomicInteger> entry : invalidMutations.entrySet())
  77. logger.info(String.format("Skipped %d mutations from unknown (probably removed) CF with id %d", entry.getValue().intValue(), entry.getKey()));
  78. // wait for all the writes to finish on the mutation stage
  79. FBUtilities.waitOnFutures(futures);
  80. logger.debug("Finished waiting on mutations from recovery");
  81. // flush replayed tables
  82. futures.clear();
  83. for (Table table : tablesRecovered)
  84. futures.addAll(table.flush());
  85. FBUtilities.waitOnFutures(futures);
  86. return replayedCount.get();
  87. }
  88. public void recover(File file) throws IOException
  89. {
  90. logger.info("Replaying " + file.getPath());
  91. final long segment = CommitLogSegment.idFromFilename(file.getName());
  92. RandomAccessReader reader = RandomAccessReader.open(new File(file.getAbsolutePath()), true);
  93. assert reader.length() <= Integer.MAX_VALUE;
  94. try
  95. {
  96. int replayPosition;
  97. if (globalPosition.segment < segment)
  98. replayPosition = 0;
  99. else if (globalPosition.segment == segment)
  100. replayPosition = globalPosition.position;
  101. else
  102. replayPosition = (int) reader.length();
  103. if (replayPosition < 0 || replayPosition >= reader.length())
  104. {
  105. // replayPosition > reader.length() can happen if some data gets flushed before it is written to the commitlog
  106. // (see https://issues.apache.org/jira/browse/CASSANDRA-2285)
  107. logger.debug("skipping replay of fully-flushed {}", file);
  108. return;
  109. }
  110. reader.seek(replayPosition);
  111. if (logger.isDebugEnabled())
  112. logger.debug("Replaying " + file + " starting at " + reader.getFilePointer());
  113. /* read the logs populate RowMutation and apply */
  114. while (!reader.isEOF())
  115. {
  116. if (logger.isDebugEnabled())
  117. logger.debug("Reading mutation at " + reader.getFilePointer());
  118. long claimedCRC32;
  119. int serializedSize;
  120. try
  121. {
  122. // any of the reads may hit EOF
  123. serializedSize = reader.readInt();
  124. if (serializedSize == CommitLog.END_OF_SEGMENT_MARKER)
  125. {
  126. logger.debug("Encountered end of segment marker at " + reader.getFilePointer());
  127. break;
  128. }
  129. // RowMutation must be at LEAST 10 bytes:
  130. // 3 each for a non-empty Table and Key (including the
  131. // 2-byte length from writeUTF/writeWithShortLength) and 4 bytes for column count.
  132. // This prevents CRC by being fooled by special-case garbage in the file; see CASSANDRA-2128
  133. if (serializedSize < 10)
  134. break;
  135. long claimedSizeChecksum = reader.readLong();
  136. checksum.reset();
  137. checksum.update(serializedSize);
  138. if (checksum.getValue() != claimedSizeChecksum)
  139. break; // entry wasn't synced correctly/fully. that's
  140. // ok.
  141. if (serializedSize > buffer.length)
  142. buffer = new byte[(int) (1.2 * serializedSize)];
  143. reader.readFully(buffer, 0, serializedSize);
  144. claimedCRC32 = reader.readLong();
  145. }
  146. catch (EOFException eof)
  147. {
  148. break; // last CL entry didn't get completely written. that's ok.
  149. }
  150. checksum.update(buffer, 0, serializedSize);
  151. if (claimedCRC32 != checksum.getValue())
  152. {
  153. // this entry must not have been fsynced. probably the rest is bad too,
  154. // but just in case there is no harm in trying them (since we still read on an entry boundary)
  155. continue;
  156. }
  157. /* deserialize the commit log entry */
  158. FastByteArrayInputStream bufIn = new FastByteArrayInputStream(buffer, 0, serializedSize);
  159. RowMutation rm;
  160. try
  161. {
  162. // assuming version here. We've gone to lengths to make sure what gets written to the CL is in
  163. // the current version. so do make sure the CL is drained prior to upgrading a node.
  164. rm = RowMutation.serializer().deserialize(new DataInputStream(bufIn), MessagingService.current_version, IColumnSerializer.Flag.LOCAL);
  165. }
  166. catch (UnknownColumnFamilyException ex)
  167. {
  168. AtomicInteger i = invalidMutations.get(ex.cfId);
  169. if (i == null)
  170. {
  171. i = new AtomicInteger(1);
  172. invalidMutations.put(ex.cfId, i);
  173. }
  174. else
  175. i.incrementAndGet();
  176. continue;
  177. }
  178. if (logger.isDebugEnabled())
  179. logger.debug(String.format("replaying mutation for %s.%s: %s", rm.getTable(), ByteBufferUtil.bytesToHex(rm.key()), "{" + StringUtils.join(rm.getColumnFamilies().iterator(), ", ")
  180. + "}"));
  181. final long entryLocation = reader.getFilePointer();
  182. final RowMutation frm = rm;
  183. Runnable runnable = new WrappedRunnable()
  184. {
  185. public void runMayThrow() throws IOException
  186. {
  187. if (Schema.instance.getKSMetaData(frm.getTable()) == null)
  188. return;
  189. if (pointInTimeExceeded(frm))
  190. return;
  191. final Table table = Table.open(frm.getTable());
  192. RowMutation newRm = new RowMutation(frm.getTable(), frm.key());
  193. // Rebuild the row mutation, omitting column families that
  194. // a) have already been flushed,
  195. // b) are part of a cf that was dropped. Keep in mind that the cf.name() is suspect. do every thing based on the cfid instead.
  196. for (ColumnFamily columnFamily : frm.getColumnFamilies())
  197. {
  198. if (Schema.instance.getCF(columnFamily.id()) == null)
  199. // null means the cf has been dropped
  200. continue;
  201. ReplayPosition rp = cfPositions.get(columnFamily.id());
  202. // replay if current segment is newer than last flushed one or,
  203. // if it is the last known segment, if we are after the replay position
  204. if (segment > rp.segment || (segment == rp.segment && entryLocation > rp.position))
  205. {
  206. newRm.add(columnFamily);
  207. replayedCount.incrementAndGet();
  208. }
  209. }
  210. if (!newRm.isEmpty())
  211. {
  212. Table.open(newRm.getTable()).apply(newRm, false);
  213. tablesRecovered.add(table);
  214. }
  215. }
  216. };
  217. futures.add(StageManager.getStage(Stage.MUTATION).submit(runnable));
  218. if (futures.size() > MAX_OUTSTANDING_REPLAY_COUNT)
  219. {
  220. FBUtilities.waitOnFutures(futures);
  221. futures.clear();
  222. }
  223. }
  224. }
  225. finally
  226. {
  227. FileUtils.closeQuietly(reader);
  228. logger.info("Finished reading " + file);
  229. }
  230. }
  231. protected boolean pointInTimeExceeded(RowMutation frm)
  232. {
  233. long restoreTarget = CommitLog.instance.archiver.restorePointInTime;
  234. for (ColumnFamily families : frm.getColumnFamilies())
  235. {
  236. if (families.maxTimestamp() > restoreTarget)
  237. return true;
  238. }
  239. return false;
  240. }
  241. }