1616 * limitations under the License.
1717 */
1818
19-
2019
2120package com .dtstack .flink .sql .sink .hbase ;
2221
22+ import com .dtstack .flink .sql .enums .EUpdateMode ;
2323import com .dtstack .flink .sql .sink .MetricOutputFormat ;
2424import com .google .common .collect .Lists ;
2525import org .apache .commons .lang3 .StringUtils ;
3131import org .apache .hadoop .hbase .TableName ;
3232import org .apache .hadoop .hbase .client .Connection ;
3333import org .apache .hadoop .hbase .client .ConnectionFactory ;
34+ import org .apache .hadoop .hbase .client .Delete ;
3435import org .apache .hadoop .hbase .client .Put ;
3536import org .apache .hadoop .hbase .client .Table ;
37+ import org .apache .hadoop .hbase .util .Bytes ;
3638import org .slf4j .Logger ;
3739import org .slf4j .LoggerFactory ;
40+
3841import java .io .IOException ;
3942import java .text .SimpleDateFormat ;
4043import java .util .List ;
@@ -54,8 +57,9 @@ public class HbaseOutputFormat extends MetricOutputFormat<Tuple2> {
5457 private String [] rowkey ;
5558 private String tableName ;
5659 private String [] columnNames ;
60+ private String updateMode ;
5761 private String [] columnTypes ;
58- private Map <String ,String > columnNameFamily ;
62+ private Map <String , String > columnNameFamily ;
5963
6064 private String [] families ;
6165 private String [] qualifiers ;
@@ -75,7 +79,7 @@ public void configure(Configuration parameters) {
7579 LOG .warn ("---configure---" );
7680 conf = HBaseConfiguration .create ();
7781 conf .set ("hbase.zookeeper.quorum" , host );
78- if (zkParent != null && !"" .equals (zkParent )){
82+ if (zkParent != null && !"" .equals (zkParent )) {
7983 conf .set ("zookeeper.znode.parent" , zkParent );
8084 }
8185 LOG .warn ("---configure end ---" );
@@ -91,38 +95,23 @@ public void open(int taskNumber, int numTasks) throws IOException {
9195 }
9296
9397 @ Override
94- public void writeRecord (Tuple2 tuple2 ) {
95-
98+ public void writeRecord (Tuple2 tuple2 ) {
9699 Tuple2 <Boolean , Row > tupleTrans = tuple2 ;
97100 Boolean retract = tupleTrans .getField (0 );
98- if (!retract ){
99- //FIXME 暂时不处理hbase删除操作--->hbase要求有key,所有认为都是可以执行update查找
100- return ;
101+ if (!retract && StringUtils .equalsIgnoreCase (updateMode , EUpdateMode .UPSERT .name ())) {
102+ dealDelete (tupleTrans );
103+ } else {
104+ dealInsert (tupleTrans );
101105 }
106+ }
102107
108+ protected void dealInsert (Tuple2 <Boolean , Row > tupleTrans ) {
103109 Row record = tupleTrans .getField (1 );
104- List <String > rowKeyValues = getRowKeyValues (record );
105- // all rowkey not null
106- if (rowKeyValues .size () != rowkey .length ) {
107- LOG .error ("row key value must not null,record is .." , record );
108- outDirtyRecords .inc ();
110+ Put put = getPutByRow (record );
111+ if (put == null ) {
109112 return ;
110113 }
111114
112- String key = StringUtils .join (rowKeyValues , "-" );
113- Put put = new Put (key .getBytes ());
114- for (int i = 0 ; i < record .getArity (); ++i ) {
115- Object fieldVal = record .getField (i );
116- if (fieldVal == null ) {
117- continue ;
118- }
119- byte [] val = fieldVal .toString ().getBytes ();
120- byte [] cf = families [i ].getBytes ();
121- byte [] qualifier = qualifiers [i ].getBytes ();
122-
123- put .addColumn (cf , qualifier , val );
124- }
125-
126115 try {
127116 table .put (put );
128117 } catch (IOException e ) {
@@ -137,14 +126,65 @@ public void writeRecord(Tuple2 tuple2) {
137126 LOG .info (record .toString ());
138127 }
139128 outRecords .inc ();
129+ }
140130
131+ protected void dealDelete (Tuple2 <Boolean , Row > tupleTrans ) {
132+ Row record = tupleTrans .getField (1 );
133+ String rowKey = buildRowKey (record );
134+ if (!StringUtils .isEmpty (rowKey )) {
135+ Delete delete = new Delete (Bytes .toBytes (rowKey ));
136+ try {
137+ table .delete (delete );
138+ } catch (IOException e ) {
139+ outDirtyRecords .inc ();
140+ if (outDirtyRecords .getCount () % dirtyDataPrintFrequency == 0 || LOG .isDebugEnabled ()) {
141+ LOG .error ("record insert failed .." , record .toString ());
142+ LOG .error ("" , e );
143+ }
144+ }
145+ if (outRecords .getCount () % rowLenth == 0 ) {
146+ LOG .info (record .toString ());
147+ }
148+ outRecords .inc ();
149+ }
150+ }
151+
152+ private Put getPutByRow (Row record ) {
153+ String rowKey = buildRowKey (record );
154+ if (StringUtils .isEmpty (rowKey )) {
155+ return null ;
156+ }
157+ Put put = new Put (rowKey .getBytes ());
158+ for (int i = 0 ; i < record .getArity (); ++i ) {
159+ Object fieldVal = record .getField (i );
160+ if (fieldVal == null ) {
161+ continue ;
162+ }
163+ byte [] val = fieldVal .toString ().getBytes ();
164+ byte [] cf = families [i ].getBytes ();
165+ byte [] qualifier = qualifiers [i ].getBytes ();
166+
167+ put .addColumn (cf , qualifier , val );
168+ }
169+ return put ;
170+ }
171+
172+ private String buildRowKey (Row record ) {
173+ List <String > rowKeyValues = getRowKeyValues (record );
174+ // all rowkey not null
175+ if (rowKeyValues .size () != rowkey .length ) {
176+ LOG .error ("row key value must not null,record is .." , record );
177+ outDirtyRecords .inc ();
178+ return "" ;
179+ }
180+ return StringUtils .join (rowKeyValues , "-" );
141181 }
142182
143183 private List <String > getRowKeyValues (Row record ) {
144184 List <String > rowKeyValues = Lists .newArrayList ();
145185 for (int i = 0 ; i < rowkey .length ; ++i ) {
146186 String colName = rowkey [i ];
147- int rowKeyIndex = 0 ; //rowkey index
187+ int rowKeyIndex = 0 ;
148188 for (; rowKeyIndex < columnNames .length ; ++rowKeyIndex ) {
149189 if (columnNames [rowKeyIndex ].equals (colName )) {
150190 break ;
@@ -168,13 +208,14 @@ private List<String> getRowKeyValues(Row record) {
168208
169209 @ Override
170210 public void close () throws IOException {
171- if (conn != null ) {
211+ if (conn != null ) {
172212 conn .close ();
173213 conn = null ;
174214 }
175215 }
176216
177- private HbaseOutputFormat () {}
217+ private HbaseOutputFormat () {
218+ }
178219
179220 public static HbaseOutputFormatBuilder buildHbaseOutputFormat () {
180221 return new HbaseOutputFormatBuilder ();
@@ -193,7 +234,7 @@ public HbaseOutputFormatBuilder setHost(String host) {
193234 return this ;
194235 }
195236
196- public HbaseOutputFormatBuilder setZkParent (String parent ){
237+ public HbaseOutputFormatBuilder setZkParent (String parent ) {
197238 format .zkParent = parent ;
198239 return this ;
199240 }
@@ -209,6 +250,11 @@ public HbaseOutputFormatBuilder setRowkey(String[] rowkey) {
209250 return this ;
210251 }
211252
253+ public HbaseOutputFormatBuilder setUpdateMode (String updateMode ) {
254+ format .updateMode = updateMode ;
255+ return this ;
256+ }
257+
212258 public HbaseOutputFormatBuilder setColumnNames (String [] columnNames ) {
213259 format .columnNames = columnNames ;
214260 return this ;
0 commit comments