View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.snapshot;
20  
21  import java.io.InputStream;
22  import java.io.IOException;
23  import java.io.OutputStream;
24  import java.util.Arrays;
25  import java.util.HashMap;
26  import java.util.HashSet;
27  import java.util.LinkedList;
28  import java.util.List;
29  import java.util.Map;
30  import java.util.Set;
31  import java.util.TreeMap;
32  
33  import org.apache.commons.logging.Log;
34  import org.apache.commons.logging.LogFactory;
35  import org.apache.hadoop.classification.InterfaceAudience;
36  import org.apache.hadoop.conf.Configuration;
37  import org.apache.hadoop.fs.FileStatus;
38  import org.apache.hadoop.fs.FileSystem;
39  import org.apache.hadoop.fs.Path;
40  import org.apache.hadoop.hbase.HColumnDescriptor;
41  import org.apache.hadoop.hbase.HRegionInfo;
42  import org.apache.hadoop.hbase.HTableDescriptor;
43  import org.apache.hadoop.hbase.backup.HFileArchiver;
44  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
45  import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
46  import org.apache.hadoop.hbase.io.HFileLink;
47  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
48  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
49  import org.apache.hadoop.hbase.regionserver.HRegion;
50  import org.apache.hadoop.hbase.regionserver.StoreFile;
51  import org.apache.hadoop.hbase.util.Bytes;
52  import org.apache.hadoop.hbase.util.FSUtils;
53  import org.apache.hadoop.hbase.util.FSVisitor;
54  import org.apache.hadoop.hbase.util.ModifyRegionUtils;
55  import org.apache.hadoop.io.IOUtils;
56  
57  /**
58   * Helper to Restore/Clone a Snapshot
59   *
60   * <p>The helper assumes that a table is already created, and by calling restore()
61   * the content present in the snapshot will be restored as the new content of the table.
62   *
63   * <p>Clone from Snapshot: If the target table is empty, the restore operation
64   * is just a "clone operation", where the only operations are:
65   * <ul>
66   *  <li>for each region in the snapshot create a new region
67   *    (note that the region will have a different name, since the encoding contains the table name)
68   *  <li>for each file in the region create a new HFileLink to point to the original file.
69   *  <li>restore the logs, if any
70   * </ul>
71   *
72   * <p>Restore from Snapshot:
73   * <ul>
74   *  <li>for each region in the table verify which are available in the snapshot and which are not
75   *    <ul>
76   *    <li>if the region is not present in the snapshot, remove it.
77   *    <li>if the region is present in the snapshot
78   *      <ul>
79   *      <li>for each file in the table region verify which are available in the snapshot
80   *        <ul>
81   *          <li>if the hfile is not present in the snapshot, remove it
82   *          <li>if the hfile is present, keep it (nothing to do)
83   *        </ul>
84   *      <li>for each file in the snapshot region but not in the table
85   *        <ul>
86   *          <li>create a new HFileLink that point to the original file
87   *        </ul>
88   *      </ul>
89   *    </ul>
90   *  <li>for each region in the snapshot not present in the current table state
91   *    <ul>
92   *    <li>create a new region and for each file in the region create a new HFileLink
93   *      (This is the same as the clone operation)
94   *    </ul>
95   *  <li>restore the logs, if any
96   * </ul>
97   */
98  @InterfaceAudience.Private
99  public class RestoreSnapshotHelper {
100   private static final Log LOG = LogFactory.getLog(RestoreSnapshotHelper.class);
101 
102   private final Map<byte[], byte[]> regionsMap =
103         new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
104 
105   private final ForeignExceptionDispatcher monitor;
106   private final MonitoredTask status;
107 
108   private final SnapshotDescription snapshotDesc;
109   private final Path snapshotDir;
110 
111   private final HTableDescriptor tableDesc;
112   private final Path tableDir;
113 
114   private final Configuration conf;
115   private final FileSystem fs;
116 
117   public RestoreSnapshotHelper(final Configuration conf, final FileSystem fs,
118       final SnapshotDescription snapshotDescription, final Path snapshotDir,
119       final HTableDescriptor tableDescriptor, final Path tableDir,
120       final ForeignExceptionDispatcher monitor, final MonitoredTask status)
121   {
122     this.fs = fs;
123     this.conf = conf;
124     this.snapshotDesc = snapshotDescription;
125     this.snapshotDir = snapshotDir;
126     this.tableDesc = tableDescriptor;
127     this.tableDir = tableDir;
128     this.monitor = monitor;
129     this.status = status;
130   }
131 
132   /**
133    * Restore the on-disk table to a specified snapshot state.
134    * @return the set of regions touched by the restore operation
135    */
136   public RestoreMetaChanges restoreHdfsRegions() throws IOException {
137     LOG.debug("starting restore");
138     Set<String> snapshotRegionNames = SnapshotReferenceUtil.getSnapshotRegionNames(fs, snapshotDir);
139     if (snapshotRegionNames == null) {
140       LOG.warn("Nothing to restore. Snapshot " + snapshotDesc + " looks empty");
141       return null;
142     }
143 
144     RestoreMetaChanges metaChanges = new RestoreMetaChanges();
145 
146     // Identify which region are still available and which not.
147     // NOTE: we rely upon the region name as: "table name, start key, end key"
148     List<HRegionInfo> tableRegions = getTableRegions();
149     if (tableRegions != null) {
150       monitor.rethrowException();
151       for (HRegionInfo regionInfo: tableRegions) {
152         String regionName = regionInfo.getEncodedName();
153         if (snapshotRegionNames.contains(regionName)) {
154           LOG.info("region to restore: " + regionName);
155           snapshotRegionNames.remove(regionName);
156           metaChanges.addRegionToRestore(regionInfo);
157         } else {
158           LOG.info("region to remove: " + regionName);
159           metaChanges.addRegionToRemove(regionInfo);
160         }
161       }
162 
163       // Restore regions using the snapshot data
164       monitor.rethrowException();
165       status.setStatus("Restoring table regions...");
166       restoreHdfsRegions(metaChanges.getRegionsToRestore());
167       status.setStatus("Finished restoring all table regions.");
168 
169       // Remove regions from the current table
170       monitor.rethrowException();
171       status.setStatus("Starting to delete excess regions from table");
172       removeHdfsRegions(metaChanges.getRegionsToRemove());
173       status.setStatus("Finished deleting excess regions from table.");
174     }
175 
176     // Regions to Add: present in the snapshot but not in the current table
177     if (snapshotRegionNames.size() > 0) {
178       List<HRegionInfo> regionsToAdd = new LinkedList<HRegionInfo>();
179 
180       monitor.rethrowException();
181       for (String regionName: snapshotRegionNames) {
182         LOG.info("region to add: " + regionName);
183         Path regionDir = new Path(snapshotDir, regionName);
184         regionsToAdd.add(HRegion.loadDotRegionInfoFileContent(fs, regionDir));
185       }
186 
187       // Create new regions cloning from the snapshot
188       monitor.rethrowException();
189       status.setStatus("Cloning regions...");
190       HRegionInfo[] clonedRegions = cloneHdfsRegions(regionsToAdd);
191       metaChanges.setNewRegions(clonedRegions);
192       status.setStatus("Finished cloning regions.");
193     }
194 
195     // Restore WALs
196     monitor.rethrowException();
197     status.setStatus("Restoring WALs to table...");
198     restoreWALs();
199     status.setStatus("Finished restoring WALs to table.");
200 
201     return metaChanges;
202   }
203 
204   /**
205    * Describe the set of operations needed to update META after restore.
206    */
207   public static class RestoreMetaChanges {
208     private List<HRegionInfo> regionsToRestore = null;
209     private List<HRegionInfo> regionsToRemove = null;
210     private List<HRegionInfo> regionsToAdd = null;
211 
212     /**
213      * @return true if there're new regions
214      */
215     public boolean hasRegionsToAdd() {
216       return this.regionsToAdd != null && this.regionsToAdd.size() > 0;
217     }
218 
219     /**
220      * Returns the list of new regions added during the on-disk restore.
221      * The caller is responsible to add the regions to META.
222      * e.g MetaEditor.addRegionsToMeta(...)
223      * @return the list of regions to add to META
224      */
225     public List<HRegionInfo> getRegionsToAdd() {
226       return this.regionsToAdd;
227     }
228 
229     /**
230      * @return true if there're regions to restore
231      */
232     public boolean hasRegionsToRestore() {
233       return this.regionsToRestore != null && this.regionsToRestore.size() > 0;
234     }
235 
236     /**
237      * Returns the list of 'restored regions' during the on-disk restore.
238      * The caller is responsible to add the regions to META if not present.
239      * @return the list of regions restored
240      */
241     public List<HRegionInfo> getRegionsToRestore() {
242       return this.regionsToRestore;
243     }
244 
245     /**
246      * @return true if there're regions to remove
247      */
248     public boolean hasRegionsToRemove() {
249       return this.regionsToRemove != null && this.regionsToRemove.size() > 0;
250     }
251 
252     /**
253      * Returns the list of regions removed during the on-disk restore.
254      * The caller is responsible to remove the regions from META.
255      * e.g. MetaEditor.deleteRegions(...)
256      * @return the list of regions to remove from META
257      */
258     public List<HRegionInfo> getRegionsToRemove() {
259       return this.regionsToRemove;
260     }
261 
262     void setNewRegions(final HRegionInfo[] hris) {
263       if (hris != null) {
264         regionsToAdd = Arrays.asList(hris);
265       } else {
266         regionsToAdd = null;
267       }
268     }
269 
270     void addRegionToRemove(final HRegionInfo hri) {
271       if (regionsToRemove == null) {
272         regionsToRemove = new LinkedList<HRegionInfo>();
273       }
274       regionsToRemove.add(hri);
275     }
276 
277     void addRegionToRestore(final HRegionInfo hri) {
278       if (regionsToRestore == null) {
279         regionsToRestore = new LinkedList<HRegionInfo>();
280       }
281       regionsToRestore.add(hri);
282     }
283   }
284 
285   /**
286    * Remove specified regions from the file-system, using the archiver.
287    */
288   private void removeHdfsRegions(final List<HRegionInfo> regions) throws IOException {
289     if (regions != null && regions.size() > 0) {
290       for (HRegionInfo hri: regions) {
291         HFileArchiver.archiveRegion(conf, fs, hri);
292       }
293     }
294   }
295 
296   /**
297    * Restore specified regions by restoring content to the snapshot state.
298    */
299   private void restoreHdfsRegions(final List<HRegionInfo> regions) throws IOException {
300     if (regions == null || regions.size() == 0) return;
301     for (HRegionInfo hri: regions) restoreRegion(hri);
302   }
303 
304   /**
305    * Restore region by removing files not in the snapshot
306    * and adding the missing ones from the snapshot.
307    */
308   private void restoreRegion(HRegionInfo regionInfo) throws IOException {
309     Path snapshotRegionDir = new Path(snapshotDir, regionInfo.getEncodedName());
310     Map<String, List<String>> snapshotFiles =
311                 SnapshotReferenceUtil.getRegionHFileReferences(fs, snapshotRegionDir);
312     Path regionDir = new Path(tableDir, regionInfo.getEncodedName());
313     String tableName = tableDesc.getNameAsString();
314 
315     // Restore families present in the table
316     for (Path familyDir: FSUtils.getFamilyDirs(fs, regionDir)) {
317       byte[] family = Bytes.toBytes(familyDir.getName());
318       Set<String> familyFiles = getTableRegionFamilyFiles(familyDir);
319       List<String> snapshotFamilyFiles = snapshotFiles.remove(familyDir.getName());
320       if (snapshotFamilyFiles != null) {
321         List<String> hfilesToAdd = new LinkedList<String>();
322         for (String hfileName: snapshotFamilyFiles) {
323           if (familyFiles.contains(hfileName)) {
324             // HFile already present
325             familyFiles.remove(hfileName);
326           } else {
327             // HFile missing
328             hfilesToAdd.add(hfileName);
329           }
330         }
331 
332         // Restore Missing files
333         for (String hfileName: hfilesToAdd) {
334           LOG.trace("Adding HFileLink " + hfileName +
335             " to region=" + regionInfo.getEncodedName() + " table=" + tableName);
336           restoreStoreFile(familyDir, regionInfo, hfileName);
337         }
338 
339         // Remove hfiles not present in the snapshot
340         for (String hfileName: familyFiles) {
341           Path hfile = new Path(familyDir, hfileName);
342           LOG.trace("Removing hfile=" + hfile +
343             " from region=" + regionInfo.getEncodedName() + " table=" + tableName);
344           HFileArchiver.archiveStoreFile(fs, regionInfo, conf, tableDir, family, hfile);
345         }
346       } else {
347         // Family doesn't exists in the snapshot
348         LOG.trace("Removing family=" + Bytes.toString(family) +
349           " from region=" + regionInfo.getEncodedName() + " table=" + tableName);
350         HFileArchiver.archiveFamily(fs, conf, regionInfo, tableDir, family);
351         fs.delete(familyDir, true);
352       }
353     }
354 
355     // Add families not present in the table
356     for (Map.Entry<String, List<String>> familyEntry: snapshotFiles.entrySet()) {
357       Path familyDir = new Path(regionDir, familyEntry.getKey());
358       if (!fs.mkdirs(familyDir)) {
359         throw new IOException("Unable to create familyDir=" + familyDir);
360       }
361 
362       for (String hfileName: familyEntry.getValue()) {
363         LOG.trace("Adding HFileLink " + hfileName + " to table=" + tableName);
364         restoreStoreFile(familyDir, regionInfo, hfileName);
365       }
366     }
367   }
368 
369   /**
370    * @return The set of files in the specified family directory.
371    */
372   private Set<String> getTableRegionFamilyFiles(final Path familyDir) throws IOException {
373     Set<String> familyFiles = new HashSet<String>();
374 
375     FileStatus[] hfiles = FSUtils.listStatus(fs, familyDir);
376     if (hfiles == null) return familyFiles;
377 
378     for (FileStatus hfileRef: hfiles) {
379       String hfileName = hfileRef.getPath().getName();
380       familyFiles.add(hfileName);
381     }
382 
383     return familyFiles;
384   }
385 
386   /**
387    * Clone specified regions. For each region create a new region
388    * and create a HFileLink for each hfile.
389    */
390   private HRegionInfo[] cloneHdfsRegions(final List<HRegionInfo> regions) throws IOException {
391     if (regions == null || regions.size() == 0) return null;
392 
393     final Map<String, HRegionInfo> snapshotRegions =
394       new HashMap<String, HRegionInfo>(regions.size());
395 
396     // clone region info (change embedded tableName with the new one)
397     HRegionInfo[] clonedRegionsInfo = new HRegionInfo[regions.size()];
398     for (int i = 0; i < clonedRegionsInfo.length; ++i) {
399       // clone the region info from the snapshot region info
400       HRegionInfo snapshotRegionInfo = regions.get(i);
401       clonedRegionsInfo[i] = cloneRegionInfo(snapshotRegionInfo);
402 
403       // add the region name mapping between snapshot and cloned
404       String snapshotRegionName = snapshotRegionInfo.getEncodedName();
405       String clonedRegionName = clonedRegionsInfo[i].getEncodedName();
406       regionsMap.put(Bytes.toBytes(snapshotRegionName), Bytes.toBytes(clonedRegionName));
407       LOG.info("clone region=" + snapshotRegionName + " as " + clonedRegionName);
408 
409       // Add mapping between cloned region name and snapshot region info
410       snapshotRegions.put(clonedRegionName, snapshotRegionInfo);
411     }
412 
413     // create the regions on disk
414     ModifyRegionUtils.createRegions(conf, tableDir.getParent(),
415       tableDesc, clonedRegionsInfo, new ModifyRegionUtils.RegionFillTask() {
416         public void fillRegion(final HRegion region) throws IOException {
417           cloneRegion(region, snapshotRegions.get(region.getRegionInfo().getEncodedName()));
418         }
419       });
420 
421     return clonedRegionsInfo;
422   }
423 
424   /**
425    * Clone region directory content from the snapshot info.
426    *
427    * Each region is encoded with the table name, so the cloned region will have
428    * a different region name.
429    *
430    * Instead of copying the hfiles a HFileLink is created.
431    *
432    * @param region {@link HRegion} cloned
433    * @param snapshotRegionInfo
434    */
435   private void cloneRegion(final HRegion region, final HRegionInfo snapshotRegionInfo)
436       throws IOException {
437     final Path snapshotRegionDir = new Path(snapshotDir, snapshotRegionInfo.getEncodedName());
438     final Path regionDir = new Path(tableDir, region.getRegionInfo().getEncodedName());
439     final String tableName = tableDesc.getNameAsString();
440     SnapshotReferenceUtil.visitRegionStoreFiles(fs, snapshotRegionDir,
441       new FSVisitor.StoreFileVisitor() {
442         public void storeFile (final String region, final String family, final String hfile)
443             throws IOException {
444           LOG.info("Adding HFileLink " + hfile + " to table=" + tableName);
445           Path familyDir = new Path(regionDir, family);
446           restoreStoreFile(familyDir, snapshotRegionInfo, hfile);
447         }
448     });
449   }
450 
451   /**
452    * Create a new {@link HFileLink} to reference the store file.
453    * <p>The store file in the snapshot can be a simple hfile, an HFileLink or a reference.
454    * <ul>
455    *   <li>hfile: abc -> table=region-abc
456    *   <li>reference: abc.1234 -> table=region-abc.1234
457    *   <li>hfilelink: table=region-hfile -> table=region-hfile
458    * </ul>
459    * @param familyDir destination directory for the store file
460    * @param regionInfo destination region info for the table
461    * @param hfileName store file name (can be a Reference, HFileLink or simple HFile)
462    */
463   private void restoreStoreFile(final Path familyDir, final HRegionInfo regionInfo,
464       final String hfileName) throws IOException {
465     if (HFileLink.isHFileLink(hfileName)) {
466       HFileLink.createFromHFileLink(conf, fs, familyDir, hfileName);
467     } else if (StoreFile.isReference(hfileName)) {
468       restoreReferenceFile(familyDir, regionInfo, hfileName);
469     } else {
470       HFileLink.create(conf, fs, familyDir, regionInfo, hfileName);
471     }
472   }
473 
474   /**
475    * Create a new {@link Reference} as copy of the source one.
476    * <p><blockquote><pre>
477    * The source table looks like:
478    *    1234/abc      (original file)
479    *    5678/abc.1234 (reference file)
480    *
481    * After the clone operation looks like:
482    *   wxyz/table=1234-abc
483    *   stuv/table=1234-abc.wxyz
484    *
485    * NOTE that the region name in the clone changes (md5 of regioninfo)
486    * and the reference should reflect that change.
487    * </pre></blockquote>
488    * @param familyDir destination directory for the store file
489    * @param regionInfo destination region info for the table
490    * @param hfileName reference file name
491    */
492   private void restoreReferenceFile(final Path familyDir, final HRegionInfo regionInfo,
493       final String hfileName) throws IOException {
494     // Extract the referred information (hfile name and parent region)
495     String tableName = snapshotDesc.getTable();
496     Path refPath = StoreFile.getReferredToFile(new Path(new Path(new Path(tableName,
497         regionInfo.getEncodedName()), familyDir.getName()), hfileName));
498     String snapshotRegionName = refPath.getParent().getParent().getName();
499     String fileName = refPath.getName();
500 
501     // The new reference should have the cloned region name as parent, if it is a clone.
502     String clonedRegionName = Bytes.toString(regionsMap.get(Bytes.toBytes(snapshotRegionName)));
503     if (clonedRegionName == null) clonedRegionName = snapshotRegionName;
504 
505     // The output file should be a reference link table=snapshotRegion-fileName.clonedRegionName
506     String refLink = fileName;
507     if (!HFileLink.isHFileLink(fileName)) {
508       refLink = HFileLink.createHFileLinkName(tableName, snapshotRegionName, fileName);
509     }
510     Path outPath = new Path(familyDir, refLink + '.' + clonedRegionName);
511 
512     // Create the new reference
513     Path linkPath = new Path(familyDir,
514       HFileLink.createHFileLinkName(tableName, regionInfo.getEncodedName(), hfileName));
515     InputStream in = new HFileLink(conf, linkPath).open(fs);
516     OutputStream out = fs.create(outPath);
517     IOUtils.copyBytes(in, out, conf);
518   }
519 
520   /**
521    * Create a new {@link HRegionInfo} from the snapshot region info.
522    * Keep the same startKey, endKey, regionId and split information but change
523    * the table name.
524    *
525    * @param snapshotRegionInfo Info for region to clone.
526    * @return the new HRegion instance
527    */
528   public HRegionInfo cloneRegionInfo(final HRegionInfo snapshotRegionInfo) {
529     return new HRegionInfo(tableDesc.getName(),
530                       snapshotRegionInfo.getStartKey(), snapshotRegionInfo.getEndKey(),
531                       snapshotRegionInfo.isSplit(), snapshotRegionInfo.getRegionId());
532   }
533 
534   /**
535    * Restore snapshot WALs.
536    *
537    * Global Snapshot keep a reference to region servers logs present during the snapshot.
538    * (/hbase/.snapshot/snapshotName/.logs/hostName/logName)
539    *
540    * Since each log contains different tables data, logs must be split to
541    * extract the table that we are interested in.
542    */
543   private void restoreWALs() throws IOException {
544     final SnapshotLogSplitter logSplitter = new SnapshotLogSplitter(conf, fs, tableDir,
545                                 Bytes.toBytes(snapshotDesc.getTable()), regionsMap);
546     try {
547       // Recover.Edits
548       SnapshotReferenceUtil.visitRecoveredEdits(fs, snapshotDir,
549           new FSVisitor.RecoveredEditsVisitor() {
550         public void recoveredEdits (final String region, final String logfile) throws IOException {
551           Path path = SnapshotReferenceUtil.getRecoveredEdits(snapshotDir, region, logfile);
552           logSplitter.splitRecoveredEdit(path);
553         }
554       });
555 
556       // Region Server Logs
557       SnapshotReferenceUtil.visitLogFiles(fs, snapshotDir, new FSVisitor.LogFileVisitor() {
558         public void logFile (final String server, final String logfile) throws IOException {
559           logSplitter.splitLog(server, logfile);
560         }
561       });
562     } finally {
563       logSplitter.close();
564     }
565   }
566 
567   /**
568    * @return the set of the regions contained in the table
569    */
570   private List<HRegionInfo> getTableRegions() throws IOException {
571     LOG.debug("get table regions: " + tableDir);
572     FileStatus[] regionDirs = FSUtils.listStatus(fs, tableDir, new FSUtils.RegionDirFilter(fs));
573     if (regionDirs == null) return null;
574 
575     List<HRegionInfo> regions = new LinkedList<HRegionInfo>();
576     for (FileStatus regionDir: regionDirs) {
577       HRegionInfo hri = HRegion.loadDotRegionInfoFileContent(fs, regionDir.getPath());
578       regions.add(hri);
579     }
580     LOG.debug("found " + regions.size() + " regions for table=" + tableDesc.getNameAsString());
581     return regions;
582   }
583 
584   /**
585    * Create a new table descriptor cloning the snapshot table schema.
586    *
587    * @param snapshotTableDescriptor
588    * @param tableName
589    * @return cloned table descriptor
590    * @throws IOException
591    */
592   public static HTableDescriptor cloneTableSchema(final HTableDescriptor snapshotTableDescriptor,
593       final byte[] tableName) throws IOException {
594     HTableDescriptor htd = new HTableDescriptor(tableName);
595     for (HColumnDescriptor hcd: snapshotTableDescriptor.getColumnFamilies()) {
596       htd.addFamily(hcd);
597     }
598     for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e:
599         snapshotTableDescriptor.getValues().entrySet()) {
600       htd.setValue(e.getKey(), e.getValue());
601     }
602     return htd;
603   }
604 }