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.coprocessor;
20  
21  import static org.junit.Assert.assertEquals;
22  import static org.junit.Assert.assertTrue;
23  
24  import java.io.IOException;
25  import java.util.ArrayList;
26  import java.util.Arrays;
27  import java.util.Collection;
28  import java.util.Collections;
29  import java.util.HashSet;
30  import java.util.List;
31  import java.util.Set;
32  import java.util.concurrent.CountDownLatch;
33  import java.util.concurrent.atomic.AtomicInteger;
34  import java.util.concurrent.atomic.AtomicLong;
35  
36  import org.apache.hadoop.hbase.util.ByteStringer;
37  import org.apache.hadoop.conf.Configuration;
38  import org.apache.hadoop.hbase.Cell;
39  import org.apache.hadoop.hbase.CellUtil;
40  import org.apache.hadoop.hbase.HBaseTestingUtility;
41  import org.apache.hadoop.hbase.HConstants;
42  import org.apache.hadoop.hbase.KeyValue;
43  import org.apache.hadoop.hbase.testclassification.MediumTests;
44  import org.apache.hadoop.hbase.client.Delete;
45  import org.apache.hadoop.hbase.client.Get;
46  import org.apache.hadoop.hbase.client.HTable;
47  import org.apache.hadoop.hbase.client.IsolationLevel;
48  import org.apache.hadoop.hbase.client.Mutation;
49  import org.apache.hadoop.hbase.client.Put;
50  import org.apache.hadoop.hbase.client.Scan;
51  import org.apache.hadoop.hbase.client.coprocessor.RowProcessorClient;
52  import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos;
53  import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.FriendsOfFriendsProcessorRequest;
54  import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.FriendsOfFriendsProcessorResponse;
55  import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.IncCounterProcessorRequest;
56  import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.IncCounterProcessorResponse;
57  import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.RowSwapProcessorRequest;
58  import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.RowSwapProcessorResponse;
59  import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.TimeoutProcessorRequest;
60  import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.TimeoutProcessorResponse;
61  import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
62  import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessRequest;
63  import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessResponse;
64  import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.RowProcessorService;
65  import org.apache.hadoop.hbase.regionserver.BaseRowProcessor;
66  import org.apache.hadoop.hbase.regionserver.HRegion;
67  import org.apache.hadoop.hbase.regionserver.InternalScanner;
68  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
69  import org.apache.hadoop.hbase.util.Bytes;
70  import org.junit.AfterClass;
71  import org.junit.BeforeClass;
72  import org.junit.Test;
73  import org.junit.experimental.categories.Category;
74  
75  import com.google.protobuf.Message;
76  import com.sun.org.apache.commons.logging.Log;
77  import com.sun.org.apache.commons.logging.LogFactory;
78  
79  /**
80   * Verifies ProcessEndpoint works.
81   * The tested RowProcessor performs two scans and a read-modify-write.
82   */
83  @Category(MediumTests.class)
84  public class TestRowProcessorEndpoint {
85  
86    static final Log LOG = LogFactory.getLog(TestRowProcessorEndpoint.class);
87  
88    private static final byte[] TABLE = Bytes.toBytes("testtable");
89    private final static byte[] ROW = Bytes.toBytes("testrow");
90    private final static byte[] ROW2 = Bytes.toBytes("testrow2");
91    private final static byte[] FAM = Bytes.toBytes("friendlist");
92  
93    // Column names
94    private final static byte[] A = Bytes.toBytes("a");
95    private final static byte[] B = Bytes.toBytes("b");
96    private final static byte[] C = Bytes.toBytes("c");
97    private final static byte[] D = Bytes.toBytes("d");
98    private final static byte[] E = Bytes.toBytes("e");
99    private final static byte[] F = Bytes.toBytes("f");
100   private final static byte[] G = Bytes.toBytes("g");
101   private final static byte[] COUNTER = Bytes.toBytes("counter");
102   private final static AtomicLong myTimer = new AtomicLong(0);
103   private final AtomicInteger failures = new AtomicInteger(0);
104 
105   private static HBaseTestingUtility util = new HBaseTestingUtility();
106   private static volatile int expectedCounter = 0;
107   private static int rowSize, row2Size;
108 
109   private volatile static HTable table = null;
110   private volatile static boolean swapped = false;
111   private volatile CountDownLatch startSignal;
112   private volatile CountDownLatch doneSignal;
113 
114   @BeforeClass
115   public static void setupBeforeClass() throws Exception {
116     Configuration conf = util.getConfiguration();
117     conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
118         RowProcessorEndpoint.class.getName());
119     conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
120     conf.setLong("hbase.hregion.row.processor.timeout", 1000L);
121     util.startMiniCluster();
122   }
123 
124   @AfterClass
125   public static void tearDownAfterClass() throws Exception {
126     util.shutdownMiniCluster();
127   }
128 
129   public void prepareTestData() throws Exception {
130     try {
131       util.getHBaseAdmin().disableTable(TABLE);
132       util.getHBaseAdmin().deleteTable(TABLE);
133     } catch (Exception e) {
134       // ignore table not found
135     }
136     table = util.createTable(TABLE, FAM);
137     {
138       Put put = new Put(ROW);
139       put.add(FAM, A, Bytes.add(B, C));    // B, C are friends of A
140       put.add(FAM, B, Bytes.add(D, E, F)); // D, E, F are friends of B
141       put.add(FAM, C, G);                  // G is a friend of C
142       table.put(put);
143       rowSize = put.size();
144     }
145     Put put = new Put(ROW2);
146     put.add(FAM, D, E);
147     put.add(FAM, F, G);
148     table.put(put);
149     row2Size = put.size();
150   }
151 
152   @Test
153   public void testDoubleScan() throws Throwable {
154     prepareTestData();
155 
156     CoprocessorRpcChannel channel = table.coprocessorService(ROW);
157     RowProcessorEndpoint.FriendsOfFriendsProcessor processor =
158         new RowProcessorEndpoint.FriendsOfFriendsProcessor(ROW, A);
159     RowProcessorService.BlockingInterface service =
160         RowProcessorService.newBlockingStub(channel);
161     ProcessRequest request = RowProcessorClient.getRowProcessorPB(processor);
162     ProcessResponse protoResult = service.process(null, request);
163     FriendsOfFriendsProcessorResponse response =
164         FriendsOfFriendsProcessorResponse.parseFrom(protoResult.getRowProcessorResult());
165     Set<String> result = new HashSet<String>();
166     result.addAll(response.getResultList());
167     Set<String> expected =
168       new HashSet<String>(Arrays.asList(new String[]{"d", "e", "f", "g"}));
169     Get get = new Get(ROW);
170     LOG.debug("row keyvalues:" + stringifyKvs(table.get(get).listCells()));
171     assertEquals(expected, result);
172   }
173 
174   @Test
175   public void testReadModifyWrite() throws Throwable {
176     prepareTestData();
177     failures.set(0);
178     int numThreads = 100;
179     concurrentExec(new IncrementRunner(), numThreads);
180     Get get = new Get(ROW);
181     LOG.debug("row keyvalues:" + stringifyKvs(table.get(get).listCells()));
182     int finalCounter = incrementCounter(table);
183     assertEquals(numThreads + 1, finalCounter);
184     assertEquals(0, failures.get());
185   }
186 
187   class IncrementRunner implements Runnable {
188     @Override
189     public void run() {
190       try {
191         incrementCounter(table);
192       } catch (Throwable e) {
193         e.printStackTrace();
194       }
195     }
196   }
197 
198   private int incrementCounter(HTable table) throws Throwable {
199     CoprocessorRpcChannel channel = table.coprocessorService(ROW);
200     RowProcessorEndpoint.IncrementCounterProcessor processor =
201         new RowProcessorEndpoint.IncrementCounterProcessor(ROW);
202     RowProcessorService.BlockingInterface service =
203         RowProcessorService.newBlockingStub(channel);
204     ProcessRequest request = RowProcessorClient.getRowProcessorPB(processor);
205     ProcessResponse protoResult = service.process(null, request);
206     IncCounterProcessorResponse response = IncCounterProcessorResponse
207         .parseFrom(protoResult.getRowProcessorResult());
208     Integer result = response.getResponse();
209     return result;
210   }
211 
212   private void concurrentExec(
213       final Runnable task, final int numThreads) throws Throwable {
214     startSignal = new CountDownLatch(numThreads);
215     doneSignal = new CountDownLatch(numThreads);
216     for (int i = 0; i < numThreads; ++i) {
217       new Thread(new Runnable() {
218         @Override
219         public void run() {
220           try {
221             startSignal.countDown();
222             startSignal.await();
223             task.run();
224           } catch (Throwable e) {
225             failures.incrementAndGet();
226             e.printStackTrace();
227           }
228           doneSignal.countDown();
229         }
230       }).start();
231     }
232     doneSignal.await();
233   }
234 
235   @Test
236   public void testMultipleRows() throws Throwable {
237     prepareTestData();
238     failures.set(0);
239     int numThreads = 100;
240     concurrentExec(new SwapRowsRunner(), numThreads);
241     LOG.debug("row keyvalues:" +
242               stringifyKvs(table.get(new Get(ROW)).listCells()));
243     LOG.debug("row2 keyvalues:" +
244               stringifyKvs(table.get(new Get(ROW2)).listCells()));
245     assertEquals(rowSize, table.get(new Get(ROW)).listCells().size());
246     assertEquals(row2Size, table.get(new Get(ROW2)).listCells().size());
247     assertEquals(0, failures.get());
248   }
249 
250   class SwapRowsRunner implements Runnable {
251     @Override
252     public void run() {
253       try {
254         swapRows(table);
255       } catch (Throwable e) {
256         e.printStackTrace();
257       }
258     }
259   }
260 
261   private void swapRows(HTable table) throws Throwable {
262     CoprocessorRpcChannel channel = table.coprocessorService(ROW);
263     RowProcessorEndpoint.RowSwapProcessor processor =
264         new RowProcessorEndpoint.RowSwapProcessor(ROW, ROW2);
265     RowProcessorService.BlockingInterface service =
266         RowProcessorService.newBlockingStub(channel);
267     ProcessRequest request = RowProcessorClient.getRowProcessorPB(processor);
268     service.process(null, request);
269   }
270 
271   @Test
272   public void testTimeout() throws Throwable {
273     prepareTestData();
274     CoprocessorRpcChannel channel = table.coprocessorService(ROW);
275     RowProcessorEndpoint.TimeoutProcessor processor =
276         new RowProcessorEndpoint.TimeoutProcessor(ROW);
277     RowProcessorService.BlockingInterface service =
278         RowProcessorService.newBlockingStub(channel);
279     ProcessRequest request = RowProcessorClient.getRowProcessorPB(processor);
280     boolean exceptionCaught = false;
281     try {
282       service.process(null, request);
283     } catch (Exception e) {
284       exceptionCaught = true;
285     }
286     assertTrue(exceptionCaught);
287   }
288 
289   /**
290    * This class defines two RowProcessors:
291    * IncrementCounterProcessor and FriendsOfFriendsProcessor.
292    *
293    * We define the RowProcessors as the inner class of the endpoint.
294    * So they can be loaded with the endpoint on the coprocessor.
295    */
296   public static class RowProcessorEndpoint<S extends Message,T extends Message>
297   extends BaseRowProcessorEndpoint<S,T> implements CoprocessorService {
298     public static class IncrementCounterProcessor extends
299         BaseRowProcessor<IncrementCounterProcessorTestProtos.IncCounterProcessorRequest,
300         IncrementCounterProcessorTestProtos.IncCounterProcessorResponse> {
301       int counter = 0;
302       byte[] row = new byte[0];
303 
304       /**
305        * Empty constructor for Writable
306        */
307       IncrementCounterProcessor() {
308       }
309 
310       IncrementCounterProcessor(byte[] row) {
311         this.row = row;
312       }
313 
314       @Override
315       public Collection<byte[]> getRowsToLock() {
316         return Collections.singleton(row);
317       }
318 
319       @Override
320       public IncCounterProcessorResponse getResult() {
321         IncCounterProcessorResponse.Builder i = IncCounterProcessorResponse.newBuilder();
322         i.setResponse(counter);
323         return i.build();
324       }
325 
326       @Override
327       public boolean readOnly() {
328         return false;
329       }
330 
331       @Override
332       public void process(long now, HRegion region,
333           List<Mutation> mutations, WALEdit walEdit) throws IOException {
334         // Scan current counter
335         List<Cell> kvs = new ArrayList<Cell>();
336         Scan scan = new Scan(row, row);
337         scan.addColumn(FAM, COUNTER);
338         doScan(region, scan, kvs);
339         counter = kvs.size() == 0 ? 0 :
340           Bytes.toInt(CellUtil.cloneValue(kvs.iterator().next()));
341 
342         // Assert counter value
343         assertEquals(expectedCounter, counter);
344 
345         // Increment counter and send it to both memstore and wal edit
346         counter += 1;
347         expectedCounter += 1;
348 
349 
350         Put p = new Put(row);
351         KeyValue kv =
352             new KeyValue(row, FAM, COUNTER, now, Bytes.toBytes(counter));
353         p.add(kv);
354         mutations.add(p);
355         walEdit.add(kv);
356 
357         // We can also inject some meta data to the walEdit
358         KeyValue metaKv = new KeyValue(
359             row, WALEdit.METAFAMILY,
360             Bytes.toBytes("I just increment counter"),
361             Bytes.toBytes(counter));
362         walEdit.add(metaKv);
363       }
364 
365       @Override
366       public IncCounterProcessorRequest getRequestData() throws IOException {
367         IncCounterProcessorRequest.Builder builder = IncCounterProcessorRequest.newBuilder();
368         builder.setCounter(counter);
369         builder.setRow(ByteStringer.wrap(row));
370         return builder.build();
371       }
372 
373       @Override
374       public void initialize(IncCounterProcessorRequest msg) {
375         this.row = msg.getRow().toByteArray();
376         this.counter = msg.getCounter();
377       }
378     }
379 
380     public static class FriendsOfFriendsProcessor extends
381         BaseRowProcessor<FriendsOfFriendsProcessorRequest, FriendsOfFriendsProcessorResponse> {
382       byte[] row = null;
383       byte[] person = null;
384       final Set<String> result = new HashSet<String>();
385 
386       /**
387        * Empty constructor for Writable
388        */
389       FriendsOfFriendsProcessor() {
390       }
391 
392       FriendsOfFriendsProcessor(byte[] row, byte[] person) {
393         this.row = row;
394         this.person = person;
395       }
396 
397       @Override
398       public Collection<byte[]> getRowsToLock() {
399         return Collections.singleton(row);
400       }
401 
402       @Override
403       public FriendsOfFriendsProcessorResponse getResult() {
404         FriendsOfFriendsProcessorResponse.Builder builder = 
405             FriendsOfFriendsProcessorResponse.newBuilder();
406         builder.addAllResult(result);
407         return builder.build();
408       }
409 
410       @Override
411       public boolean readOnly() {
412         return true;
413       }
414 
415       @Override
416       public void process(long now, HRegion region,
417           List<Mutation> mutations, WALEdit walEdit) throws IOException {
418         List<Cell> kvs = new ArrayList<Cell>();
419         { // First scan to get friends of the person
420           Scan scan = new Scan(row, row);
421           scan.addColumn(FAM, person);
422           doScan(region, scan, kvs);
423         }
424 
425         // Second scan to get friends of friends
426         Scan scan = new Scan(row, row);
427         for (Cell kv : kvs) {
428           byte[] friends = CellUtil.cloneValue(kv);
429           for (byte f : friends) {
430             scan.addColumn(FAM, new byte[]{f});
431           }
432         }
433         doScan(region, scan, kvs);
434 
435         // Collect result
436         result.clear();
437         for (Cell kv : kvs) {
438           for (byte b : CellUtil.cloneValue(kv)) {
439             result.add((char)b + "");
440           }
441         }
442       }
443 
444       @Override
445       public FriendsOfFriendsProcessorRequest getRequestData() throws IOException {
446         FriendsOfFriendsProcessorRequest.Builder builder =
447             FriendsOfFriendsProcessorRequest.newBuilder();
448         builder.setPerson(ByteStringer.wrap(person));
449         builder.setRow(ByteStringer.wrap(row));
450         builder.addAllResult(result);
451         FriendsOfFriendsProcessorRequest f = builder.build();
452         return f;
453       }
454 
455       @Override
456       public void initialize(FriendsOfFriendsProcessorRequest request) 
457           throws IOException {
458         this.person = request.getPerson().toByteArray();
459         this.row = request.getRow().toByteArray();
460         result.clear();
461         result.addAll(request.getResultList());
462       }
463     }
464 
465     public static class RowSwapProcessor extends
466         BaseRowProcessor<RowSwapProcessorRequest, RowSwapProcessorResponse> {
467       byte[] row1 = new byte[0];
468       byte[] row2 = new byte[0];
469 
470       /**
471        * Empty constructor for Writable
472        */
473       RowSwapProcessor() {
474       }
475 
476       RowSwapProcessor(byte[] row1, byte[] row2) {
477         this.row1 = row1;
478         this.row2 = row2;
479       }
480 
481       @Override
482       public Collection<byte[]> getRowsToLock() {
483         List<byte[]> rows = new ArrayList<byte[]>();
484         rows.add(row1);
485         rows.add(row2);
486         return rows;
487       }
488 
489       @Override
490       public boolean readOnly() {
491         return false;
492       }
493 
494       @Override
495       public RowSwapProcessorResponse getResult() {
496         return RowSwapProcessorResponse.getDefaultInstance();
497       }
498 
499       @Override
500       public void process(long now, HRegion region,
501           List<Mutation> mutations, WALEdit walEdit) throws IOException {
502 
503         // Override the time to avoid race-condition in the unit test caused by
504         // inacurate timer on some machines
505         now = myTimer.getAndIncrement();
506 
507         // Scan both rows
508         List<Cell> kvs1 = new ArrayList<Cell>();
509         List<Cell> kvs2 = new ArrayList<Cell>();
510         doScan(region, new Scan(row1, row1), kvs1);
511         doScan(region, new Scan(row2, row2), kvs2);
512 
513         // Assert swapped
514         if (swapped) {
515           assertEquals(rowSize, kvs2.size());
516           assertEquals(row2Size, kvs1.size());
517         } else {
518           assertEquals(rowSize, kvs1.size());
519           assertEquals(row2Size, kvs2.size());
520         }
521         swapped = !swapped;
522 
523         // Add and delete keyvalues
524         List<List<Cell>> kvs = new ArrayList<List<Cell>>();
525         kvs.add(kvs1);
526         kvs.add(kvs2);
527         byte[][] rows = new byte[][]{row1, row2};
528         for (int i = 0; i < kvs.size(); ++i) {
529           for (Cell kv : kvs.get(i)) {
530             // Delete from the current row and add to the other row
531             Delete d = new Delete(rows[i]);
532             KeyValue kvDelete =
533                 new KeyValue(rows[i], CellUtil.cloneFamily(kv), CellUtil.cloneQualifier(kv), 
534                     kv.getTimestamp(), KeyValue.Type.Delete);
535             d.addDeleteMarker(kvDelete);
536             Put p = new Put(rows[1 - i]);
537             KeyValue kvAdd =
538                 new KeyValue(rows[1 - i], CellUtil.cloneFamily(kv), CellUtil.cloneQualifier(kv),
539                     now, CellUtil.cloneValue(kv));
540             p.add(kvAdd);
541             mutations.add(d);
542             walEdit.add(kvDelete);
543             mutations.add(p);
544             walEdit.add(kvAdd);
545           }
546         }
547       }
548 
549       @Override
550       public String getName() {
551         return "swap";
552       }
553 
554       @Override
555       public RowSwapProcessorRequest getRequestData() throws IOException {
556         RowSwapProcessorRequest.Builder builder = RowSwapProcessorRequest.newBuilder();
557         builder.setRow1(ByteStringer.wrap(row1));
558         builder.setRow2(ByteStringer.wrap(row2));
559         return builder.build();
560       }
561 
562       @Override
563       public void initialize(RowSwapProcessorRequest msg) {
564         this.row1 = msg.getRow1().toByteArray();
565         this.row2 = msg.getRow2().toByteArray();
566       }
567     }
568 
569     public static class TimeoutProcessor extends
570         BaseRowProcessor<TimeoutProcessorRequest, TimeoutProcessorResponse> {
571 
572       byte[] row = new byte[0];
573 
574       /**
575        * Empty constructor for Writable
576        */
577       public TimeoutProcessor() {
578       }
579 
580       public TimeoutProcessor(byte[] row) {
581         this.row = row;
582       }
583 
584       public Collection<byte[]> getRowsToLock() {
585         return Collections.singleton(row);
586       }
587 
588       @Override
589       public TimeoutProcessorResponse getResult() {
590         return TimeoutProcessorResponse.getDefaultInstance();
591       }
592 
593       @Override
594       public void process(long now, HRegion region,
595           List<Mutation> mutations, WALEdit walEdit) throws IOException {
596         try {
597           // Sleep for a long time so it timeout
598           Thread.sleep(100 * 1000L);
599         } catch (Exception e) {
600           throw new IOException(e);
601         }
602       }
603 
604       @Override
605       public boolean readOnly() {
606         return true;
607       }
608 
609       @Override
610       public String getName() {
611         return "timeout";
612       }
613 
614       @Override
615       public TimeoutProcessorRequest getRequestData() throws IOException {
616         TimeoutProcessorRequest.Builder builder = TimeoutProcessorRequest.newBuilder();
617         builder.setRow(ByteStringer.wrap(row));
618         return builder.build();
619       }
620 
621       @Override
622       public void initialize(TimeoutProcessorRequest msg) throws IOException {
623         this.row = msg.getRow().toByteArray();
624       }
625     }
626 
627     public static void doScan(
628         HRegion region, Scan scan, List<Cell> result) throws IOException {
629       InternalScanner scanner = null;
630       try {
631         scan.setIsolationLevel(IsolationLevel.READ_UNCOMMITTED);
632         scanner = region.getScanner(scan);
633         result.clear();
634         scanner.next(result);
635       } finally {
636         if (scanner != null) scanner.close();
637       }
638     }
639   }
640 
641   static String stringifyKvs(Collection<Cell> kvs) {
642     StringBuilder out = new StringBuilder();
643     out.append("[");
644     if (kvs != null) {
645       for (Cell kv : kvs) {
646         byte[] col = CellUtil.cloneQualifier(kv);
647         byte[] val = CellUtil.cloneValue(kv);
648         if (Bytes.equals(col, COUNTER)) {
649           out.append(Bytes.toStringBinary(col) + ":" +
650                      Bytes.toInt(val) + " ");
651         } else {
652           out.append(Bytes.toStringBinary(col) + ":" +
653                      Bytes.toStringBinary(val) + " ");
654         }
655       }
656     }
657     out.append("]");
658     return out.toString();
659   }
660 
661 }