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  
21  package org.apache.hadoop.hbase.rest.client;
22  
23  import java.io.IOException;
24  import java.util.ArrayList;
25  import java.util.Collection;
26  import java.util.Iterator;
27  import java.util.List;
28  import java.util.Map;
29  import java.util.Set;
30  import java.util.TreeMap;
31  
32  import org.apache.commons.logging.Log;
33  import org.apache.commons.logging.LogFactory;
34  
35  import org.apache.hadoop.hbase.client.*;
36  import org.apache.hadoop.hbase.client.coprocessor.Batch;
37  import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
38  import org.apache.hadoop.util.StringUtils;
39  
40  import org.apache.hadoop.conf.Configuration;
41  import org.apache.hadoop.hbase.HBaseConfiguration;
42  import org.apache.hadoop.hbase.HConstants;
43  import org.apache.hadoop.hbase.HTableDescriptor;
44  import org.apache.hadoop.hbase.KeyValue;
45  import org.apache.hadoop.hbase.client.RowMutations;
46  import org.apache.hadoop.hbase.client.Delete;
47  import org.apache.hadoop.hbase.client.Get;
48  import org.apache.hadoop.hbase.client.HTableInterface;
49  import org.apache.hadoop.hbase.client.Increment;
50  import org.apache.hadoop.hbase.client.Put;
51  import org.apache.hadoop.hbase.client.Row;
52  import org.apache.hadoop.hbase.client.Result;
53  import org.apache.hadoop.hbase.client.ResultScanner;
54  import org.apache.hadoop.hbase.client.RowLock;
55  import org.apache.hadoop.hbase.client.Scan;
56  import org.apache.hadoop.hbase.io.TimeRange;
57  import org.apache.hadoop.hbase.rest.Constants;
58  import org.apache.hadoop.hbase.rest.model.CellModel;
59  import org.apache.hadoop.hbase.rest.model.CellSetModel;
60  import org.apache.hadoop.hbase.rest.model.RowModel;
61  import org.apache.hadoop.hbase.rest.model.ScannerModel;
62  import org.apache.hadoop.hbase.rest.model.TableSchemaModel;
63  import org.apache.hadoop.hbase.util.Bytes;
64  
65  /**
66   * HTable interface to remote tables accessed via REST gateway
67   */
68  public class RemoteHTable implements HTableInterface {
69  
70    private static final Log LOG = LogFactory.getLog(RemoteHTable.class);
71  
72    final Client client;
73    final Configuration conf;
74    final byte[] name;
75    final int maxRetries;
76    final long sleepTime;
77  
78    @SuppressWarnings("rawtypes")
79    protected String buildRowSpec(final byte[] row, final Map familyMap, 
80        final long startTime, final long endTime, final int maxVersions) {
81      StringBuffer sb = new StringBuffer();
82      sb.append('/');
83      sb.append(Bytes.toStringBinary(name));
84      sb.append('/');
85      sb.append(Bytes.toStringBinary(row));
86      Set families = familyMap.entrySet();
87      if (families != null) {
88        Iterator i = familyMap.entrySet().iterator();
89        if (i.hasNext()) {
90          sb.append('/');
91        }
92        while (i.hasNext()) {
93          Map.Entry e = (Map.Entry)i.next();
94          Collection quals = (Collection)e.getValue();
95          if (quals != null && !quals.isEmpty()) {
96            Iterator ii = quals.iterator();
97            while (ii.hasNext()) {
98              sb.append(Bytes.toStringBinary((byte[])e.getKey()));
99              sb.append(':');
100             Object o = ii.next();
101             // Puts use byte[] but Deletes use KeyValue
102             if (o instanceof byte[]) {
103               sb.append(Bytes.toStringBinary((byte[])o));
104             } else if (o instanceof KeyValue) {
105               sb.append(Bytes.toStringBinary(((KeyValue)o).getQualifier()));
106             } else {
107               throw new RuntimeException("object type not handled");
108             }
109             if (ii.hasNext()) {
110               sb.append(',');
111             }
112           }
113         } else {
114           sb.append(Bytes.toStringBinary((byte[])e.getKey()));
115           sb.append(':');
116         }
117         if (i.hasNext()) {
118           sb.append(',');
119         }
120       }
121     }
122     if (startTime != 0 && endTime != Long.MAX_VALUE) {
123       sb.append('/');
124       sb.append(startTime);
125       if (startTime != endTime) {
126         sb.append(',');
127         sb.append(endTime);
128       }
129     } else if (endTime != Long.MAX_VALUE) {
130       sb.append('/');
131       sb.append(endTime);
132     }
133     if (maxVersions > 1) {
134       sb.append("?v=");
135       sb.append(maxVersions);
136     }
137     return sb.toString();
138   }
139 
140   protected String buildMultiRowSpec(final byte[][] rows, int maxVersions) {
141     StringBuilder sb = new StringBuilder();
142     sb.append('/');
143     sb.append(Bytes.toStringBinary(name));
144     sb.append("/multiget/");
145     if (rows == null || rows.length == 0) {
146       return sb.toString();
147     }
148     sb.append("?");
149     for(int i=0; i<rows.length; i++) {
150       byte[] rk = rows[i];
151       if (i != 0) {
152         sb.append('&');
153       }
154       sb.append("row=");
155       sb.append(Bytes.toStringBinary(rk));
156     }
157     sb.append("&v=");
158     sb.append(maxVersions);
159 
160     return sb.toString();
161   }
162 
163   protected Result[] buildResultFromModel(final CellSetModel model) {
164     List<Result> results = new ArrayList<Result>();
165     for (RowModel row: model.getRows()) {
166       List<KeyValue> kvs = new ArrayList<KeyValue>();
167       for (CellModel cell: row.getCells()) {
168         byte[][] split = KeyValue.parseColumn(cell.getColumn());
169         byte[] column = split[0];
170         byte[] qualifier = split.length > 1 ? split[1] : null;
171         kvs.add(new KeyValue(row.getKey(), column, qualifier, 
172           cell.getTimestamp(), cell.getValue()));
173       }
174       results.add(new Result(kvs));
175     }
176     return results.toArray(new Result[results.size()]);
177   }
178 
179   protected CellSetModel buildModelFromPut(Put put) {
180     RowModel row = new RowModel(put.getRow());
181     long ts = put.getTimeStamp();
182     for (List<KeyValue> kvs: put.getFamilyMap().values()) {
183       for (KeyValue kv: kvs) {
184         row.addCell(new CellModel(kv.getFamily(), kv.getQualifier(),
185           ts != HConstants.LATEST_TIMESTAMP ? ts : kv.getTimestamp(),
186           kv.getValue()));
187       }
188     }
189     CellSetModel model = new CellSetModel();
190     model.addRow(row);
191     return model;
192   }
193 
194   /**
195    * Constructor
196    * @param client
197    * @param name
198    */
199   public RemoteHTable(Client client, String name) {
200     this(client, HBaseConfiguration.create(), Bytes.toBytes(name), null);
201   }
202 
203   /**
204    * Constructor
205    * @param client
206    * @param name
207    * @param accessToken
208    * @deprecated accessToken is not used and will be removed
209    */
210   @Deprecated
211   public RemoteHTable(Client client, String name, String accessToken) {
212     this(client, HBaseConfiguration.create(), Bytes.toBytes(name), accessToken);
213   }
214 
215   /**
216    * Constructor
217    * @param client
218    * @param conf
219    * @param name
220    */
221   public RemoteHTable(Client client, Configuration conf, String name) {
222     this(client, conf, Bytes.toBytes(name), null);
223   }
224 
225   /**
226    * Constructor
227    * @param client
228    * @param conf
229    * @param name
230    * @param accessToken
231    * @deprecated accessToken is not used and will be removed
232    */
233   @Deprecated
234   public RemoteHTable(Client client, Configuration conf, String name,
235       String accessToken) {
236     this(client, conf, Bytes.toBytes(name), accessToken);
237   }
238 
239   /**
240    * Constructor
241    * @param client
242    * @param conf
243    * @param name
244    */
245   public RemoteHTable(Client client, Configuration conf, byte[] name) {
246     this(client, conf, name, null);
247   }
248 
249   /**
250    * Constructor
251    * @param client
252    * @param conf
253    * @param name
254    * @param accessToken
255    * @deprecated accessToken is not used and will be removed
256    */
257   @Deprecated
258   public RemoteHTable(Client client, Configuration conf, byte[] name,
259       String accessToken) {
260     this.client = client;
261     this.conf = conf;
262     this.name = name;
263     this.maxRetries = conf.getInt("hbase.rest.client.max.retries", 10);
264     this.sleepTime = conf.getLong("hbase.rest.client.sleep", 1000);
265   }
266 
267   public byte[] getTableName() {
268     return name.clone();
269   }
270 
271   public Configuration getConfiguration() {
272     return conf;
273   }
274 
275   public HTableDescriptor getTableDescriptor() throws IOException {
276     StringBuilder sb = new StringBuilder();
277     sb.append('/');
278     sb.append(Bytes.toStringBinary(name));
279     sb.append('/');
280     sb.append("schema");
281     for (int i = 0; i < maxRetries; i++) {
282       Response response = client.get(sb.toString(), Constants.MIMETYPE_PROTOBUF);
283       int code = response.getCode();
284       switch (code) {
285       case 200:
286         TableSchemaModel schema = new TableSchemaModel();
287         schema.getObjectFromMessage(response.getBody());
288         return schema.getTableDescriptor();
289       case 509: 
290         try {
291           Thread.sleep(sleepTime);
292         } catch (InterruptedException e) { }
293         break;
294       default:
295         throw new IOException("schema request returned " + code);
296       }
297     }
298     throw new IOException("schema request timed out");
299   }
300 
301   public void close() throws IOException {
302     client.shutdown();
303   }
304 
305   public Result get(Get get) throws IOException {
306     TimeRange range = get.getTimeRange();
307     String spec = buildRowSpec(get.getRow(), get.getFamilyMap(),
308       range.getMin(), range.getMax(), get.getMaxVersions());
309     if (get.getFilter() != null) {
310       LOG.warn("filters not supported on gets");
311     }
312     Result[] results = getResults(spec);
313     if (results.length > 0) {
314       if (results.length > 1) {
315         LOG.warn("too many results for get (" + results.length + ")");
316       }
317       return results[0];
318     } else {
319       return new Result();
320     }
321   }
322 
323   public Result[] get(List<Get> gets) throws IOException {
324     byte[][] rows = new byte[gets.size()][];
325     int maxVersions = 1;
326     int count = 0;
327 
328     for (Get g : gets) {
329 
330       if (count == 0) {
331         maxVersions = g.getMaxVersions();
332       } else if (g.getMaxVersions() != maxVersions) {
333         LOG.warn("MaxVersions on Gets do not match, using the first in the list ("
334             + maxVersions +")");
335       }
336 
337       if (g.getFilter() != null) {
338         LOG.warn("filters not supported on gets");
339       }
340 
341       rows[count] = g.getRow();
342       count++;
343     }
344 
345     String spec = buildMultiRowSpec(rows, maxVersions);
346 
347     return getResults(spec);
348   }
349 
350   private Result[] getResults(String spec) throws IOException {
351     for (int i = 0; i < maxRetries; i++) {
352       Response response = client.get(spec, Constants.MIMETYPE_PROTOBUF);
353       int code = response.getCode();
354       switch (code) {
355         case 200:
356           CellSetModel model = new CellSetModel();
357           model.getObjectFromMessage(response.getBody());
358           Result[] results = buildResultFromModel(model);
359           if (results.length > 0) {
360             return results;
361           }
362           // fall through
363         case 404:
364           return new Result[0];
365 
366         case 509:
367           try {
368             Thread.sleep(sleepTime);
369           } catch (InterruptedException e) {
370           }
371           break;
372         default:
373           throw new IOException("get request returned " + code);
374       }
375     }
376     throw new IOException("get request timed out");
377   }
378 
379   public boolean exists(Get get) throws IOException {
380     LOG.warn("exists() is really get(), just use get()");
381     Result result = get(get);
382     return (result != null && !(result.isEmpty()));
383   }
384 
385   public void put(Put put) throws IOException {
386     CellSetModel model = buildModelFromPut(put);
387     StringBuilder sb = new StringBuilder();
388     sb.append('/');
389     sb.append(Bytes.toStringBinary(name));
390     sb.append('/');
391     sb.append(Bytes.toStringBinary(put.getRow()));
392     for (int i = 0; i < maxRetries; i++) {
393       Response response = client.put(sb.toString(), Constants.MIMETYPE_PROTOBUF,
394         model.createProtobufOutput());
395       int code = response.getCode();
396       switch (code) {
397       case 200:
398         return;
399       case 509:
400         try {
401           Thread.sleep(sleepTime);
402         } catch (InterruptedException e) { }
403         break;
404       default:
405         throw new IOException("put request failed with " + code);
406       }
407     }
408     throw new IOException("put request timed out");
409   }
410 
411   public void put(List<Put> puts) throws IOException {
412     // this is a trick: The gateway accepts multiple rows in a cell set and
413     // ignores the row specification in the URI
414 
415     // separate puts by row
416     TreeMap<byte[],List<KeyValue>> map =
417       new TreeMap<byte[],List<KeyValue>>(Bytes.BYTES_COMPARATOR);
418     for (Put put: puts) {
419       byte[] row = put.getRow();
420       List<KeyValue> kvs = map.get(row);
421       if (kvs == null) {
422         kvs = new ArrayList<KeyValue>();
423         map.put(row, kvs);
424       }
425       for (List<KeyValue> l: put.getFamilyMap().values()) {
426         kvs.addAll(l);
427       }
428     }
429 
430     // build the cell set
431     CellSetModel model = new CellSetModel();
432     for (Map.Entry<byte[], List<KeyValue>> e: map.entrySet()) {
433       RowModel row = new RowModel(e.getKey());
434       for (KeyValue kv: e.getValue()) {
435         row.addCell(new CellModel(kv));
436       }
437       model.addRow(row);
438     }
439 
440     // build path for multiput
441     StringBuilder sb = new StringBuilder();
442     sb.append('/');
443     sb.append(Bytes.toStringBinary(name));
444     sb.append("/$multiput"); // can be any nonexistent row
445     for (int i = 0; i < maxRetries; i++) {
446       Response response = client.put(sb.toString(), Constants.MIMETYPE_PROTOBUF,
447         model.createProtobufOutput());
448       int code = response.getCode();
449       switch (code) {
450       case 200:
451         return;
452       case 509:
453         try {
454           Thread.sleep(sleepTime);
455         } catch (InterruptedException e) { }
456         break;
457       default:
458         throw new IOException("multiput request failed with " + code);
459       }
460     }
461     throw new IOException("multiput request timed out");
462   }
463 
464   public void delete(Delete delete) throws IOException {
465     String spec = buildRowSpec(delete.getRow(), delete.getFamilyMap(),
466       delete.getTimeStamp(), delete.getTimeStamp(), 1);
467     for (int i = 0; i < maxRetries; i++) {
468       Response response = client.delete(spec);
469       int code = response.getCode();
470       switch (code) {
471       case 200:
472         return;
473       case 509:
474         try {
475           Thread.sleep(sleepTime);
476         } catch (InterruptedException e) { }
477         break;
478       default:
479         throw new IOException("delete request failed with " + code);
480       }
481     }
482     throw new IOException("delete request timed out");
483   }
484 
485   public void delete(List<Delete> deletes) throws IOException {
486     for (Delete delete: deletes) {
487       delete(delete);
488     }
489   }
490 
491   public void flushCommits() throws IOException {
492     // no-op
493   }
494 
495   class Scanner implements ResultScanner {
496 
497     String uri;
498 
499     public Scanner(Scan scan) throws IOException {
500       ScannerModel model;
501       try {
502         model = ScannerModel.fromScan(scan);
503       } catch (Exception e) {
504         throw new IOException(e);
505       }
506       StringBuffer sb = new StringBuffer();
507       sb.append('/');
508       sb.append(Bytes.toStringBinary(name));
509       sb.append('/');
510       sb.append("scanner");
511       for (int i = 0; i < maxRetries; i++) {
512         Response response = client.post(sb.toString(),
513           Constants.MIMETYPE_PROTOBUF, model.createProtobufOutput());
514         int code = response.getCode();
515         switch (code) {
516         case 201:
517           uri = response.getLocation();
518           return;
519         case 509:
520           try {
521             Thread.sleep(sleepTime);
522           } catch (InterruptedException e) { }
523           break;
524         default:
525           throw new IOException("scan request failed with " + code);
526         }
527       }
528       throw new IOException("scan request timed out");
529     }
530 
531     @Override
532     public Result[] next(int nbRows) throws IOException {
533       StringBuilder sb = new StringBuilder(uri);
534       sb.append("?n=");
535       sb.append(nbRows);
536       for (int i = 0; i < maxRetries; i++) {
537         Response response = client.get(sb.toString(),
538           Constants.MIMETYPE_PROTOBUF);
539         int code = response.getCode();
540         switch (code) {
541         case 200:
542           CellSetModel model = new CellSetModel();
543           model.getObjectFromMessage(response.getBody());
544           return buildResultFromModel(model);
545         case 204:
546         case 206:
547           return null;
548         case 509:
549           try {
550             Thread.sleep(sleepTime);
551           } catch (InterruptedException e) { }
552           break;
553         default:
554           throw new IOException("scanner.next request failed with " + code);
555         }
556       }
557       throw new IOException("scanner.next request timed out");
558     }
559 
560     @Override
561     public Result next() throws IOException {
562       Result[] results = next(1);
563       if (results == null || results.length < 1) {
564         return null;
565       }
566       return results[0];
567     }
568     
569     class Iter implements Iterator<Result> {
570 
571       Result cache;
572 
573       public Iter() {
574         try {
575           cache = Scanner.this.next();
576         } catch (IOException e) {
577           LOG.warn(StringUtils.stringifyException(e));
578         }
579       }
580 
581       @Override
582       public boolean hasNext() {
583         return cache != null;
584       }
585 
586       @Override
587       public Result next() {
588         Result result = cache;
589         try {
590           cache = Scanner.this.next();
591         } catch (IOException e) {
592           LOG.warn(StringUtils.stringifyException(e));
593           cache = null;
594         }
595         return result;
596       }
597 
598       @Override
599       public void remove() {
600         throw new RuntimeException("remove() not supported");
601       }
602       
603     }
604 
605     @Override
606     public Iterator<Result> iterator() {
607       return new Iter();
608     }
609 
610     @Override
611     public void close() {
612       try {
613         client.delete(uri);
614       } catch (IOException e) {
615         LOG.warn(StringUtils.stringifyException(e));
616       }
617     }
618 
619   }
620 
621   public ResultScanner getScanner(Scan scan) throws IOException {
622     return new Scanner(scan);
623   }
624 
625   public ResultScanner getScanner(byte[] family) throws IOException {
626     Scan scan = new Scan();
627     scan.addFamily(family);
628     return new Scanner(scan);
629   }
630 
631   public ResultScanner getScanner(byte[] family, byte[] qualifier)
632       throws IOException {
633     Scan scan = new Scan();
634     scan.addColumn(family, qualifier);
635     return new Scanner(scan);
636   }
637 
638   public boolean isAutoFlush() {
639     return true;
640   }
641 
642   public Result getRowOrBefore(byte[] row, byte[] family) throws IOException {
643     throw new IOException("getRowOrBefore not supported");
644   }
645 
646   /**
647    * @deprecated {@link RowLock} and associated operations are deprecated
648    */
649   public RowLock lockRow(byte[] row) throws IOException {
650     throw new IOException("lockRow not implemented");
651   }
652 
653   /**
654    * @deprecated {@link RowLock} and associated operations are deprecated
655    */
656   public void unlockRow(RowLock rl) throws IOException {
657     throw new IOException("unlockRow not implemented");
658   }
659 
660   public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
661       byte[] value, Put put) throws IOException {
662     // column to check-the-value
663     put.add(new KeyValue(row, family, qualifier, value));
664 
665     CellSetModel model = buildModelFromPut(put);
666     StringBuilder sb = new StringBuilder();
667     sb.append('/');
668     sb.append(Bytes.toStringBinary(name));
669     sb.append('/');
670     sb.append(Bytes.toStringBinary(put.getRow()));
671     sb.append("?check=put");
672 
673     for (int i = 0; i < maxRetries; i++) {
674       Response response = client.put(sb.toString(),
675         Constants.MIMETYPE_PROTOBUF, model.createProtobufOutput());
676       int code = response.getCode();
677       switch (code) {
678       case 200:
679         return true;
680       case 304: // NOT-MODIFIED
681         return false;
682       case 509:
683         try {
684           Thread.sleep(sleepTime);
685         } catch (final InterruptedException e) {
686         }
687         break;
688       default:
689         throw new IOException("checkAndPut request failed with " + code);
690       }
691     }
692     throw new IOException("checkAndPut request timed out");
693   }
694 
695   public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
696       byte[] value, Delete delete) throws IOException {
697     Put put = new Put(row);
698     // column to check-the-value
699     put.add(new KeyValue(row, family, qualifier, value));
700     CellSetModel model = buildModelFromPut(put);
701     StringBuilder sb = new StringBuilder();
702     sb.append('/');
703     sb.append(Bytes.toStringBinary(name));
704     sb.append('/');
705     sb.append(Bytes.toStringBinary(row));
706     sb.append("?check=delete");
707 
708     for (int i = 0; i < maxRetries; i++) {
709       Response response = client.put(sb.toString(),
710         Constants.MIMETYPE_PROTOBUF, model.createProtobufOutput());
711       int code = response.getCode();
712       switch (code) {
713       case 200:
714         return true;
715       case 304: // NOT-MODIFIED
716         return false;
717       case 509:
718         try {
719           Thread.sleep(sleepTime);
720         } catch (final InterruptedException e) {
721         }
722         break;
723       default:
724         throw new IOException("checkAndDelete request failed with " + code);
725       }
726     }
727     throw new IOException("checkAndDelete request timed out");
728   }
729 
730   public Result increment(Increment increment) throws IOException {
731     throw new IOException("Increment not supported");
732   }
733 
734   public Result append(Append append) throws IOException {
735     throw new IOException("Append not supported");
736   }
737 
738   public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
739       long amount) throws IOException {
740     throw new IOException("incrementColumnValue not supported");
741   }
742 
743   public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
744       long amount, boolean writeToWAL) throws IOException {
745     throw new IOException("incrementColumnValue not supported");
746   }
747 
748   @Override
749   public void batch(List<? extends Row> actions, Object[] results) throws IOException {
750     throw new IOException("batch not supported");
751   }
752 
753   @Override
754   public Object[] batch(List<? extends Row> actions) throws IOException {
755     throw new IOException("batch not supported");
756   }
757 
758   @Override
759   public <T extends CoprocessorProtocol> T coprocessorProxy(Class<T> protocol,
760       byte[] row) {
761     throw new
762     UnsupportedOperationException("coprocessorProxy not implemented");
763   }
764 
765   @Override
766   public <T extends CoprocessorProtocol, R> Map<byte[], R> coprocessorExec(
767       Class<T> protocol, byte[] startKey, byte[] endKey,
768       Batch.Call<T, R> callable)
769       throws IOException, Throwable {
770     throw new UnsupportedOperationException("coprocessorExec not implemented");
771   }
772 
773   @Override
774   public <T extends CoprocessorProtocol, R> void coprocessorExec(
775       Class<T> protocol, byte[] startKey, byte[] endKey,
776       Batch.Call<T, R> callable, Batch.Callback<R> callback)
777       throws IOException, Throwable {
778     throw new UnsupportedOperationException("coprocessorExec not implemented");
779   }
780 
781   @Override
782   public void mutateRow(RowMutations rm) throws IOException {
783     throw new IOException("atomicMutation not supported");
784   }
785 
786   @Override
787   public void setAutoFlush(boolean autoFlush) {
788     throw new UnsupportedOperationException("setAutoFlush not implemented");
789   }
790 
791   @Override
792   public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
793     throw new UnsupportedOperationException("setAutoFlush not implemented");
794   }
795 
796   @Override
797   public long getWriteBufferSize() {
798     throw new UnsupportedOperationException("getWriteBufferSize not implemented");
799   }
800 
801   @Override
802   public void setWriteBufferSize(long writeBufferSize) throws IOException {
803     throw new IOException("setWriteBufferSize not supported");
804   }
805 }