This repository has been archived by the owner on Feb 26, 2024. It is now read-only.
/
HBaseTypes.java
205 lines (184 loc) · 6.77 KB
/
HBaseTypes.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
/*
* *
* * Licensed to the Apache Software Foundation (ASF) under one
* * or more contributor license agreements. See the NOTICE file
* * distributed with this work for additional information
* * regarding copyright ownership. The ASF licenses this file
* * to you under the Apache License, Version 2.0 (the
* * "License"); you may not use this file except in compliance
* * with the License. You may obtain a copy of the License at
* *
* * http://www.apache.org/licenses/LICENSE-2.0
* *
* * Unless required by applicable law or agreed to in writing, software
* * distributed under the License is distributed on an "AS IS" BASIS,
* * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* * See the License for the specific language governing permissions and
* * limitations under the License.
*
*/
package org.apache.crunch.io.hbase;
import com.google.common.collect.ImmutableList;
import org.apache.crunch.CrunchRuntimeException;
import org.apache.crunch.MapFn;
import org.apache.crunch.types.PType;
import org.apache.crunch.types.writable.Writables;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.mapreduce.MutationSerialization;
import org.apache.hadoop.hbase.mapreduce.ResultSerialization;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.serializer.Deserializer;
import org.apache.hadoop.io.serializer.Serialization;
import org.apache.hadoop.io.serializer.Serializer;
import org.apache.hadoop.util.ReflectionUtils;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
public final class HBaseTypes {
public static final PType<Put> puts() {
return Writables.derived(Put.class,
new MapInFn<Put>(Put.class, MutationSerialization.class),
new MapOutFn<Put>(Put.class, MutationSerialization.class),
Writables.bytes());
}
public static final PType<Delete> deletes() {
return Writables.derived(Delete.class,
new MapInFn<Delete>(Delete.class, MutationSerialization.class),
new MapOutFn<Delete>(Delete.class, MutationSerialization.class),
Writables.bytes());
}
public static final PType<Result> results() {
return Writables.derived(Result.class,
new MapInFn<Result>(Result.class, ResultSerialization.class),
new MapOutFn<Result>(Result.class, ResultSerialization.class),
Writables.bytes());
}
public static final PType<KeyValue> keyValues() {
return Writables.derived(KeyValue.class,
new MapFn<BytesWritable, KeyValue>() {
@Override
public KeyValue map(BytesWritable input) {
return bytesToKeyValue(input);
}
},
new MapFn<KeyValue, BytesWritable>() {
@Override
public BytesWritable map(KeyValue input) {
return keyValueToBytes(input);
}
},
Writables.writables(BytesWritable.class));
}
public static final PType<Cell> cells() {
return Writables.derived(Cell.class,
new MapFn<BytesWritable, Cell>() {
@Override
public Cell map(BytesWritable input) {
return bytesToKeyValue(input);
}
},
new MapFn<Cell, BytesWritable>() {
@Override
public BytesWritable map(Cell input) {
return keyValueToBytes(input);
}
},
Writables.writables(BytesWritable.class));
}
public static BytesWritable keyValueToBytes(Cell input) {
return keyValueToBytes(KeyValue.cloneAndAddTags(input, ImmutableList.<Tag>of()));
}
public static BytesWritable keyValueToBytes(KeyValue kv) {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream dos = new DataOutputStream(baos);
try {
KeyValue.write(kv, dos);
return new BytesWritable(baos.toByteArray());
} catch (Exception e) {
throw new CrunchRuntimeException(e);
}
}
public static KeyValue bytesToKeyValue(BytesWritable input) {
return bytesToKeyValue(input.getBytes(), 0, input.getLength());
}
public static KeyValue bytesToKeyValue(byte[] array, int offset, int limit) {
ByteArrayInputStream bais = new ByteArrayInputStream(array, offset, limit);
DataInputStream dis = new DataInputStream(bais);
try {
return KeyValue.create(dis);
} catch (IOException e) {
throw new CrunchRuntimeException(e);
}
}
private static class MapInFn<T> extends MapFn<ByteBuffer, T> {
private Class<T> clazz;
private Class<? extends Serialization> serClazz;
private transient Deserializer<T> deserializer;
public MapInFn(Class<T> clazz, Class<? extends Serialization> serClazz) {
this.clazz = clazz;
this.serClazz = serClazz;
}
@Override
public void initialize() {
this.deserializer = ReflectionUtils.newInstance(serClazz, null).getDeserializer(clazz);
if (deserializer == null) {
throw new CrunchRuntimeException("No Hadoop deserializer for class: " + clazz);
}
}
@Override
public T map(ByteBuffer bb) {
if (deserializer == null) {
initialize();
}
ByteArrayInputStream bais = new ByteArrayInputStream(bb.array(), bb.position(), bb.limit());
try {
deserializer.open(bais);
T ret = deserializer.deserialize(null);
deserializer.close();
return ret;
} catch (Exception e) {
throw new CrunchRuntimeException("Deserialization errror", e);
}
}
}
private static class MapOutFn<T> extends MapFn<T, ByteBuffer> {
private Class<T> clazz;
private Class<? extends Serialization> serClazz;
private transient Serializer<T> serializer;
public MapOutFn(Class<T> clazz, Class<? extends Serialization> serClazz) {
this.clazz = clazz;
this.serClazz = serClazz;
}
@Override
public void initialize() {
this.serializer = ReflectionUtils.newInstance(serClazz, null).getSerializer(clazz);
if (serializer == null) {
throw new CrunchRuntimeException("No Hadoop serializer for class: " + clazz);
}
}
@Override
public ByteBuffer map(T out) {
if (serializer == null) {
initialize();
}
ByteArrayOutputStream baos = new ByteArrayOutputStream();
try {
serializer.open(baos);
serializer.serialize(out);
serializer.close();
return ByteBuffer.wrap(baos.toByteArray());
} catch (Exception e) {
throw new CrunchRuntimeException("Serialization errror", e);
}
}
}
private HBaseTypes() {}
}