View Javadoc

1   /*
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.replication.regionserver;
21  
22  import org.apache.commons.logging.Log;
23  import org.apache.commons.logging.LogFactory;
24  import org.apache.hadoop.conf.Configuration;
25  import org.apache.hadoop.fs.FileStatus;
26  import org.apache.hadoop.fs.FileSystem;
27  import org.apache.hadoop.fs.Path;
28  import org.apache.hadoop.hbase.HConstants;
29  import org.apache.hadoop.hbase.HServerAddress;
30  import org.apache.hadoop.hbase.KeyValue;
31  import org.apache.hadoop.hbase.client.HConnection;
32  import org.apache.hadoop.hbase.client.HConnectionManager;
33  import org.apache.hadoop.hbase.ipc.HRegionInterface;
34  import org.apache.hadoop.hbase.regionserver.wal.HLog;
35  import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
36  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
37  import org.apache.hadoop.hbase.replication.ReplicationZookeeperWrapper;
38  import org.apache.hadoop.hbase.util.Bytes;
39  import org.apache.hadoop.hbase.util.Threads;
40  
41  import java.io.EOFException;
42  import java.io.FileNotFoundException;
43  import java.io.IOException;
44  import java.util.ArrayList;
45  import java.util.Arrays;
46  import java.util.Comparator;
47  import java.util.HashSet;
48  import java.util.List;
49  import java.util.NavigableMap;
50  import java.util.Random;
51  import java.util.Set;
52  import java.util.concurrent.CountDownLatch;
53  import java.util.concurrent.PriorityBlockingQueue;
54  import java.util.concurrent.TimeUnit;
55  import java.util.concurrent.atomic.AtomicBoolean;
56  
57  /**
58   * Class that handles the source of a replication stream.
59   * Currently does not handle more than 1 slave
60   * For each slave cluster it selects a random number of peers
61   * using a replication ratio. For example, if replication ration = 0.1
62   * and slave cluster has 100 region servers, 10 will be selected.
63   * <p/>
64   * A stream is considered down when we cannot contact a region server on the
65   * peer cluster for more than 55 seconds by default.
66   * <p/>
67   *
68   */
69  public class ReplicationSource extends Thread
70      implements ReplicationSourceInterface {
71  
72    private static final Log LOG = LogFactory.getLog(ReplicationSource.class);
73    // Queue of logs to process
74    private PriorityBlockingQueue<Path> queue;
75    // container of entries to replicate
76    private HLog.Entry[] entriesArray;
77    private HConnection conn;
78    // Helper class for zookeeper
79    private ReplicationZookeeperWrapper zkHelper;
80    private Configuration conf;
81    // ratio of region servers to chose from a slave cluster
82    private float ratio;
83    private Random random;
84    // should we replicate or not?
85    private AtomicBoolean replicating;
86    // id of the peer cluster this source replicates to
87    private String peerClusterId;
88    // The manager of all sources to which we ping back our progress
89    private ReplicationSourceManager manager;
90    // Should we stop everything?
91    private AtomicBoolean stop;
92    // List of chosen sinks (region servers)
93    private List<HServerAddress> currentPeers;
94    // How long should we sleep for each retry
95    private long sleepForRetries;
96    // Max size in bytes of entriesArray
97    private long replicationQueueSizeCapacity;
98    // Max number of entries in entriesArray
99    private int replicationQueueNbCapacity;
100   // Our reader for the current log
101   private HLog.Reader reader;
102   // Current position in the log
103   private long position = 0;
104   // Path of the current log
105   private volatile Path currentPath;
106   private FileSystem fs;
107   // id of this cluster
108   private byte clusterId;
109   // total number of edits we replicated
110   private long totalReplicatedEdits = 0;
111   // The znode we currently play with
112   private String peerClusterZnode;
113   // Indicates if this queue is recovered (and will be deleted when depleted)
114   private boolean queueRecovered;
115   // List of all the dead region servers that had this queue (if recovered)
116   private String[] deadRegionServers;
117   // Maximum number of retries before taking bold actions
118   private long maxRetriesMultiplier;
119   // Current number of entries that we need to replicate
120   private int currentNbEntries = 0;
121   // Current number of operations (Put/Delete) that we need to replicate
122   private int currentNbOperations = 0;
123   // Indicates if this particular source is running
124   private volatile boolean running = true;
125   // Metrics for this source
126   private ReplicationSourceMetrics metrics;
127 
128   /**
129    * Instantiation method used by region servers
130    *
131    * @param conf configuration to use
132    * @param fs file system to use
133    * @param manager replication manager to ping to
134    * @param stopper     the atomic boolean to use to stop the regionserver
135    * @param replicating the atomic boolean that starts/stops replication
136    * @param peerClusterZnode the name of our znode
137    * @throws IOException
138    */
139   public void init(final Configuration conf,
140                    final FileSystem fs,
141                    final ReplicationSourceManager manager,
142                    final AtomicBoolean stopper,
143                    final AtomicBoolean replicating,
144                    final String peerClusterZnode)
145       throws IOException {
146     this.stop = stopper;
147     this.conf = conf;
148     this.replicationQueueSizeCapacity =
149         this.conf.getLong("replication.source.size.capacity", 1024*1024*64);
150     this.replicationQueueNbCapacity =
151         this.conf.getInt("replication.source.nb.capacity", 25000);
152     this.entriesArray = new HLog.Entry[this.replicationQueueNbCapacity];
153     for (int i = 0; i < this.replicationQueueNbCapacity; i++) {
154       this.entriesArray[i] = new HLog.Entry();
155     }
156     this.maxRetriesMultiplier =
157         this.conf.getLong("replication.source.maxretriesmultiplier", 10);
158     this.queue =
159         new PriorityBlockingQueue<Path>(
160             conf.getInt("hbase.regionserver.maxlogs", 32),
161             new LogsComparator());
162     this.conn = HConnectionManager.getConnection(conf);
163     this.zkHelper = manager.getRepZkWrapper();
164     this.ratio = this.conf.getFloat("replication.source.ratio", 0.1f);
165     this.currentPeers = new ArrayList<HServerAddress>();
166     this.random = new Random();
167     this.replicating = replicating;
168     this.manager = manager;
169     this.sleepForRetries =
170         this.conf.getLong("replication.source.sleepforretries", 1000);
171     this.fs = fs;
172     this.clusterId = Byte.valueOf(zkHelper.getClusterId());
173     this.metrics = new ReplicationSourceMetrics(peerClusterZnode);
174 
175     // Finally look if this is a recovered queue
176     this.checkIfQueueRecovered(peerClusterZnode);
177   }
178 
179   // The passed znode will be either the id of the peer cluster or
180   // the handling story of that queue in the form of id-servername-*
181   private void checkIfQueueRecovered(String peerClusterZnode) {
182     String[] parts = peerClusterZnode.split("-");
183     this.queueRecovered = parts.length != 1;
184     this.peerClusterId = this.queueRecovered ?
185         parts[0] : peerClusterZnode;
186     this.peerClusterZnode = peerClusterZnode;
187     this.deadRegionServers = new String[parts.length-1];
188     // Extract all the places where we could find the hlogs
189     for (int i = 1; i < parts.length; i++) {
190       this.deadRegionServers[i-1] = parts[i];
191     }
192   }
193 
194   /**
195    * Select a number of peers at random using the ratio. Mininum 1.
196    */
197   private void chooseSinks() {
198     this.currentPeers.clear();
199     List<HServerAddress> addresses =
200         this.zkHelper.getPeersAddresses(peerClusterId);
201     Set<HServerAddress> setOfAddr = new HashSet<HServerAddress>();
202     int nbPeers = (int) (Math.ceil(addresses.size() * ratio));
203     LOG.info("Getting " + nbPeers +
204         " rs from peer cluster # " + peerClusterId);
205     for (int i = 0; i < nbPeers; i++) {
206       HServerAddress address;
207       // Make sure we get one address that we don't already have
208       do {
209         address = addresses.get(this.random.nextInt(addresses.size()));
210       } while (setOfAddr.contains(address));
211       LOG.info("Choosing peer " + address);
212       setOfAddr.add(address);
213     }
214     this.currentPeers.addAll(setOfAddr);
215   }
216 
217   @Override
218   public void enqueueLog(Path log) {
219     this.queue.put(log);
220     this.metrics.sizeOfLogQueue.set(queue.size());
221   }
222 
223   @Override
224   public void run() {
225     connectToPeers();
226     // We were stopped while looping to connect to sinks, just abort
227     if (this.stop.get()) {
228       return;
229     }
230     // If this is recovered, the queue is already full and the first log
231     // normally has a position (unless the RS failed between 2 logs)
232     if (this.queueRecovered) {
233       this.position = this.zkHelper.getHLogRepPosition(
234           this.peerClusterZnode, this.queue.peek().getName());
235     }
236     int sleepMultiplier = 1;
237     // Loop until we close down
238     while (!stop.get() && this.running) {
239       // Get a new path
240       if (!getNextPath()) {
241         if (sleepForRetries("No log to process", sleepMultiplier)) {
242           sleepMultiplier++;
243         }
244         continue;
245       }
246       // Open a reader on it
247       if (!openReader(sleepMultiplier)) {
248         // Reset the sleep multiplier, else it'd be reused for the next file
249         sleepMultiplier = 1;
250         continue;
251       }
252 
253       // If we got a null reader but didn't continue, then sleep and continue
254       if (this.reader == null) {
255         if (sleepForRetries("Unable to open a reader", sleepMultiplier)) {
256           sleepMultiplier++;
257         }
258         continue;
259       }
260 
261       boolean gotIOE = false;
262       currentNbEntries = 0;
263       try {
264         if(readAllEntriesToReplicateOrNextFile()) {
265           continue;
266         }
267       } catch (IOException ioe) {
268         LOG.warn(peerClusterZnode + " Got: ", ioe);
269         gotIOE = true;
270         if (ioe.getCause() instanceof EOFException) {
271 
272           boolean considerDumping = false;
273           if (this.queueRecovered) {
274             try {
275               FileStatus stat = this.fs.getFileStatus(this.currentPath);
276               if (stat.getLen() == 0) {
277                 LOG.warn(peerClusterZnode + " Got EOF and the file was empty");
278               }
279               considerDumping = true;
280             } catch (IOException e) {
281               LOG.warn(peerClusterZnode + " Got while getting file size: ", e);
282             }
283           } else if (currentNbEntries != 0) {
284             LOG.warn(peerClusterZnode + " Got EOF while reading, " +
285                 "looks like this file is broken? " + currentPath);
286             considerDumping = true;
287             currentNbEntries = 0;
288           }
289 
290           if (considerDumping &&
291               sleepMultiplier == this.maxRetriesMultiplier &&
292               processEndOfFile()) {
293             continue;
294           }
295         }
296       } finally {
297         try {
298           // if current path is null, it means we processEndOfFile hence
299           if (this.currentPath != null && !gotIOE) {
300             this.position = this.reader.getPosition();
301           }
302           if (this.reader != null) {
303             this.reader.close();
304           }
305         } catch (IOException e) {
306           gotIOE = true;
307           LOG.warn("Unable to finalize the tailing of a file", e);
308         }
309       }
310 
311       // If we didn't get anything to replicate, or if we hit a IOE,
312       // wait a bit and retry.
313       // But if we need to stop, don't bother sleeping
314       if (!stop.get() && (gotIOE || currentNbEntries == 0)) {
315         if (sleepForRetries("Nothing to replicate", sleepMultiplier)) {
316           sleepMultiplier++;
317         }
318         continue;
319       }
320       sleepMultiplier = 1;
321       shipEdits();
322 
323     }
324     LOG.debug("Source exiting " + peerClusterId);
325   }
326 
327   /**
328    * Read all the entries from the current log files and retain those
329    * that need to be replicated. Else, process the end of the current file.
330    * @return true if we got nothing and went to the next file, false if we got
331    * entries
332    * @throws IOException
333    */
334   protected boolean readAllEntriesToReplicateOrNextFile() throws IOException{
335     long seenEntries = 0;
336     if (this.position != 0) {
337       this.reader.seek(this.position);
338     }
339     HLog.Entry entry = this.reader.next(this.entriesArray[currentNbEntries]);
340     while (entry != null) {
341       WALEdit edit = entry.getEdit();
342       this.metrics.logEditsReadRate.inc(1);
343       seenEntries++;
344       // Remove all KVs that should not be replicated
345       removeNonReplicableEdits(edit);
346       HLogKey logKey = entry.getKey();
347       // Don't replicate catalog entries, if the WALEdit wasn't
348       // containing anything to replicate and if we're currently not set to replicate
349       if (!(Bytes.equals(logKey.getTablename(), HConstants.ROOT_TABLE_NAME) ||
350           Bytes.equals(logKey.getTablename(), HConstants.META_TABLE_NAME)) &&
351           edit.size() != 0 && replicating.get()) {
352         logKey.setClusterId(this.clusterId);
353         currentNbOperations += countDistinctRowKeys(edit);
354         currentNbEntries++;
355       } else {
356         this.metrics.logEditsFilteredRate.inc(1);
357       }
358       // Stop if too many entries or too big
359       if ((this.reader.getPosition() - this.position)
360           >= this.replicationQueueSizeCapacity ||
361           currentNbEntries >= this.replicationQueueNbCapacity) {
362         break;
363       }
364       entry = this.reader.next(entriesArray[currentNbEntries]);
365     }
366     LOG.debug("currentNbOperations:" + currentNbOperations +
367         " and seenEntries:" + seenEntries +
368         " and size: " + (this.reader.getPosition() - this.position));
369     // If we didn't get anything and the queue has an object, it means we
370     // hit the end of the file for sure
371     return seenEntries == 0 && processEndOfFile();
372   }
373 
374   private void connectToPeers() {
375     // Connect to peer cluster first, unless we have to stop
376     while (!this.stop.get() && this.currentPeers.size() == 0) {
377       try {
378         chooseSinks();
379         Thread.sleep(this.sleepForRetries);
380       } catch (InterruptedException e) {
381         LOG.error("Interrupted while trying to connect to sinks", e);
382       }
383     }
384   }
385 
386   /**
387    * Poll for the next path
388    * @return true if a path was obtained, false if not
389    */
390   protected boolean getNextPath() {
391     try {
392       if (this.currentPath == null) {
393         this.currentPath = queue.poll(this.sleepForRetries, TimeUnit.MILLISECONDS);
394         this.metrics.sizeOfLogQueue.set(queue.size());
395       }
396     } catch (InterruptedException e) {
397       LOG.warn("Interrupted while reading edits", e);
398     }
399     return this.currentPath != null;
400   }
401 
402   /**
403    * Open a reader on the current path
404    *
405    * @param sleepMultiplier by how many times the default sleeping time is augmented
406    * @return true if we should continue with that file, false if we are over with it
407    */
408   protected boolean openReader(int sleepMultiplier) {
409     try {
410       LOG.info("Opening log for replication " + this.currentPath.getName() +
411           " at " + this.position);
412       try {
413        this.reader = null;
414        this.reader = HLog.getReader(this.fs, this.currentPath, this.conf);
415       } catch (FileNotFoundException fnfe) {
416         if (this.queueRecovered) {
417           // We didn't find the log in the archive directory, look if it still
418           // exists in the dead RS folder (there could be a chain of failures
419           // to look at)
420           for (int i = this.deadRegionServers.length - 1; i > 0; i--) {
421             Path deadRsDirectory =
422                 new Path(this.manager.getLogDir(), this.deadRegionServers[i]);
423             Path possibleLogLocation =
424                 new Path(deadRsDirectory, currentPath.getName());
425             if (this.manager.getFs().exists(possibleLogLocation)) {
426               // We found the right new location
427               LOG.info("Log " + this.currentPath + " still exists at " +
428                   possibleLogLocation);
429               // Breaking here will make us sleep since reader is null
430               break;
431             }
432           }
433           // TODO What happens if the log was missing from every single location?
434           // Although we need to check a couple of times as the log could have
435           // been moved by the master between the checks
436         } else {
437           // If the log was archived, continue reading from there
438           Path archivedLogLocation =
439               new Path(manager.getOldLogDir(), currentPath.getName());
440           if (this.manager.getFs().exists(archivedLogLocation)) {
441             currentPath = archivedLogLocation;
442             LOG.info("Log " + this.currentPath + " was moved to " +
443                 archivedLogLocation);
444             // Open the log at the new location
445             this.openReader(sleepMultiplier);
446 
447           }
448           // TODO What happens the log is missing in both places?
449         }
450       }
451     } catch (IOException ioe) {
452       LOG.warn(peerClusterZnode + " Got: ", ioe);
453       // TODO Need a better way to determinate if a file is really gone but
454       // TODO without scanning all logs dir
455       if (sleepMultiplier == this.maxRetriesMultiplier) {
456         LOG.warn("Waited too long for this file, considering dumping");
457         return !processEndOfFile();
458       }
459     }
460     return true;
461   }
462 
463   /**
464    * Do the sleeping logic
465    * @param msg Why we sleep
466    * @param sleepMultiplier by how many times the default sleeping time is augmented
467    * @return
468    */
469   protected boolean sleepForRetries(String msg, int sleepMultiplier) {
470     try {
471       LOG.debug(msg + ", sleeping " + sleepForRetries + " times " + sleepMultiplier);
472       Thread.sleep(this.sleepForRetries * sleepMultiplier);
473     } catch (InterruptedException e) {
474       LOG.debug("Interrupted while sleeping between retries");
475     }
476     return sleepMultiplier < maxRetriesMultiplier;
477   }
478 
479   /**
480    * We only want KVs that are scoped other than local
481    * @param edit The KV to check for replication
482    */
483   protected void removeNonReplicableEdits(WALEdit edit) {
484     NavigableMap<byte[], Integer> scopes = edit.getScopes();
485     List<KeyValue> kvs = edit.getKeyValues();
486     for (int i = 0; i < edit.size(); i++) {
487       KeyValue kv = kvs.get(i);
488       // The scope will be null or empty if
489       // there's nothing to replicate in that WALEdit
490       if (scopes == null || !scopes.containsKey(kv.getFamily())) {
491         kvs.remove(i);
492         i--;
493       }
494     }
495   }
496 
497   /**
498    * Count the number of different row keys in the given edit because of
499    * mini-batching. We assume that there's at least one KV in the WALEdit.
500    * @param edit edit to count row keys from
501    * @return number of different row keys
502    */
503   private int countDistinctRowKeys(WALEdit edit) {
504     List<KeyValue> kvs = edit.getKeyValues();
505     int distinctRowKeys = 1;
506     KeyValue lastKV = kvs.get(0);
507     for (int i = 0; i < edit.size(); i++) {
508       if (!kvs.get(i).matchingRow(lastKV)) {
509         distinctRowKeys++;
510       }
511     }
512     return distinctRowKeys;
513   }
514 
515   /**
516    * Do the shipping logic
517    */
518   protected void shipEdits() {
519     int sleepMultiplier = 1;
520     while (!stop.get()) {
521       try {
522         HRegionInterface rrs = getRS();
523         LOG.debug("Replicating " + currentNbEntries);
524         rrs.replicateLogEntries(Arrays.copyOf(this.entriesArray, currentNbEntries));
525         this.manager.logPositionAndCleanOldLogs(this.currentPath,
526             this.peerClusterZnode, this.position, queueRecovered);
527         this.totalReplicatedEdits += currentNbEntries;
528         this.metrics.shippedBatchesRate.inc(1);
529         this.metrics.shippedOpsRate.inc(
530             this.currentNbOperations);
531         this.metrics.setAgeOfLastShippedOp(
532             this.entriesArray[this.entriesArray.length-1].getKey().getWriteTime());
533         LOG.debug("Replicated in total: " + this.totalReplicatedEdits);
534         break;
535 
536       } catch (IOException ioe) {
537         LOG.warn("Unable to replicate because ", ioe);
538         try {
539           boolean down;
540           do {
541             down = isSlaveDown();
542             if (down) {
543               LOG.debug("The region server we tried to ping didn't answer, " +
544                   "sleeping " + sleepForRetries + " times " + sleepMultiplier);
545               Thread.sleep(this.sleepForRetries * sleepMultiplier);
546               if (sleepMultiplier < maxRetriesMultiplier) {
547                 sleepMultiplier++;
548               } else {
549                 chooseSinks();
550               }
551             }
552           } while (!stop.get() && down);
553         } catch (InterruptedException e) {
554           LOG.debug("Interrupted while trying to contact the peer cluster");
555         }
556 
557       }
558     }
559   }
560 
561   /**
562    * If the queue isn't empty, switch to the next one
563    * Else if this is a recovered queue, it means we're done!
564    * Else we'll just continue to try reading the log file
565    * @return true if we're done with the current file, false if we should
566    * continue trying to read from it
567    */
568   protected boolean processEndOfFile() {
569     if (this.queue.size() != 0) {
570       this.currentPath = null;
571       this.position = 0;
572       return true;
573     } else if (this.queueRecovered) {
574       this.manager.closeRecoveredQueue(this);
575       this.abort();
576       return true;
577     }
578     return false;
579   }
580 
581   public void startup() {
582     String n = Thread.currentThread().getName();
583     Thread.UncaughtExceptionHandler handler =
584         new Thread.UncaughtExceptionHandler() {
585           public void uncaughtException(final Thread t, final Throwable e) {
586             LOG.fatal("Set stop flag in " + t.getName(), e);
587             abort();
588           }
589         };
590     Threads.setDaemonThreadRunning(
591         this, n + ".replicationSource," + clusterId, handler);
592   }
593 
594   /**
595    * Hastily stop the replication, then wait for shutdown
596    */
597   private void abort() {
598     LOG.info("abort");
599     this.running = false;
600     terminate();
601   }
602 
603   public void terminate() {
604     LOG.info("terminate");
605     Threads.shutdown(this, this.sleepForRetries);
606   }
607 
608   /**
609    * Get a new region server at random from this peer
610    * @return
611    * @throws IOException
612    */
613   private HRegionInterface getRS() throws IOException {
614     if (this.currentPeers.size() == 0) {
615       throw new IOException(this.peerClusterZnode + " has 0 region servers");
616     }
617     HServerAddress address =
618         currentPeers.get(random.nextInt(this.currentPeers.size()));
619     return this.conn.getHRegionConnection(address);
620   }
621 
622   /**
623    * Check if the slave is down by trying to establish a connection
624    * @return true if down, false if up
625    * @throws InterruptedException
626    */
627   public boolean isSlaveDown() throws InterruptedException {
628     final CountDownLatch latch = new CountDownLatch(1);
629     Thread pingThread = new Thread() {
630       public void run() {
631         try {
632           HRegionInterface rrs = getRS();
633           // Dummy call which should fail
634           rrs.getHServerInfo();
635           latch.countDown();
636         } catch (IOException ex) {
637           LOG.info("Slave cluster looks down: " + ex.getMessage());
638         }
639       }
640     };
641     pingThread.start();
642     // awaits returns true if countDown happened
643     boolean down = ! latch.await(this.sleepForRetries, TimeUnit.MILLISECONDS);
644     pingThread.interrupt();
645     return down;
646   }
647 
648   /**
649    * Get the id that the source is replicating to
650    *
651    * @return peer cluster id
652    */
653   public String getPeerClusterZnode() {
654     return this.peerClusterZnode;
655   }
656 
657   /**
658    * Get the path of the current HLog
659    * @return current hlog's path
660    */
661   public Path getCurrentPath() {
662     return this.currentPath;
663   }
664 
665   /**
666    * Comparator used to compare logs together based on their start time
667    */
668   public static class LogsComparator implements Comparator<Path> {
669 
670     @Override
671     public int compare(Path o1, Path o2) {
672       return Long.valueOf(getTS(o1)).compareTo(getTS(o2));
673     }
674 
675     @Override
676     public boolean equals(Object o) {
677       return true;
678     }
679 
680     /**
681      * Split a path to get the start time
682      * For example: 10.20.20.171%3A60020.1277499063250
683      * @param p path to split
684      * @return start time
685      */
686     private long getTS(Path p) {
687       String[] parts = p.getName().split("\\.");
688       return Long.parseLong(parts[parts.length-1]);
689     }
690   }
691 
692 }