1818
1919package com .tencent .cloud .oceanus .sink ;
2020
21+ import com .google .common .util .concurrent .RateLimiter ;
2122import org .apache .flink .annotation .PublicEvolving ;
2223import org .apache .flink .configuration .ConfigOption ;
2324import org .apache .flink .configuration .ConfigOptions ;
25+ import org .apache .flink .configuration .Configuration ;
2426import org .apache .flink .configuration .ReadableConfig ;
25- import org .apache .flink .streaming .api .functions .sink .SinkFunction ;
27+ import org .apache .flink .streaming .api .functions .sink .RichSinkFunction ;
2628import org .apache .flink .table .api .TableSchema ;
2729import org .apache .flink .table .catalog .CatalogTable ;
2830import org .apache .flink .table .connector .ChangelogMode ;
3234import org .apache .flink .table .factories .DynamicTableSinkFactory ;
3335import org .apache .flink .table .factories .FactoryUtil ;
3436import org .apache .flink .table .types .DataType ;
35- import org .apache .flink .table .types .logical .LogicalType ;
3637import org .apache .flink .table .utils .TableSchemaUtils ;
3738import org .apache .flink .types .RowKind ;
38- import org .apache .flink .util .StringUtils ;
3939import org .slf4j .Logger ;
4040import org .slf4j .LoggerFactory ;
4141
4242import java .lang .reflect .Method ;
43- import java .util .Arrays ;
4443import java .util .HashSet ;
4544import java .util .Set ;
4645
47- import static org . apache . flink . table . data . RowData . createFieldGetter ;
46+ import static com . tencent . cloud . oceanus . sink . LoggerTableSinkFactory . MUTE_OUTPUT ;
4847
4948/**
5049 * Logger table sink factory prints all input records using SLF4J loggers.
@@ -57,17 +56,30 @@ public class LoggerTableSinkFactory implements DynamicTableSinkFactory {
5756
5857 public static final String IDENTIFIER = "logger" ;
5958 public static final ConfigOption <String > PRINT_IDENTIFIER = ConfigOptions
60- .key ("print-identifier" )
61- .stringType ()
62- .defaultValue ("" )
63- .withDescription ("Message that identify logger and is prefixed to the output of the value." );
59+ .key ("print-identifier" )
60+ .stringType ()
61+ .defaultValue ("" )
62+ .withDescription ("Message that identify logger and is prefixed to the output of the value." );
6463
6564 public static final ConfigOption <Boolean > ALL_CHANGELOG_MODE = ConfigOptions
6665 .key ("all-changelog-mode" )
6766 .booleanType ()
6867 .defaultValue (false )
6968 .withDescription ("Whether to accept all changelog mode." );
7069
70+ public static final ConfigOption <Integer > RECORDS_PER_SECOND = ConfigOptions
71+ .key ("records-per-second" )
72+ .intType ()
73+ .defaultValue (-1 )
74+ .withDescription ("Control how many records are written to the sink per second. " +
75+ "Use -1 for unlimited output." );
76+
77+ public static final ConfigOption <Boolean > MUTE_OUTPUT = ConfigOptions
78+ .key ("mute-output" )
79+ .booleanType ()
80+ .defaultValue (false )
81+ .withDescription ("Whether to discard all incoming records (similar to blackhole sink)." );
82+
7183 @ Override
7284 public String factoryIdentifier () {
7385 return IDENTIFIER ;
@@ -83,6 +95,8 @@ public Set<ConfigOption<?>> optionalOptions() {
8395 Set <ConfigOption <?>> optionalOptions = new HashSet <>();
8496 optionalOptions .add (PRINT_IDENTIFIER );
8597 optionalOptions .add (ALL_CHANGELOG_MODE );
98+ optionalOptions .add (RECORDS_PER_SECOND );
99+ optionalOptions .add (MUTE_OUTPUT );
86100 return optionalOptions ;
87101 }
88102
@@ -102,12 +116,15 @@ public DynamicTableSink createDynamicTableSink(Context context) {
102116 LOGGER .error (e .getMessage (), e );
103117 }
104118 TableSchema physicalSchema =
105- TableSchemaUtils .getPhysicalSchema (table .getSchema ());
119+ TableSchemaUtils .getPhysicalSchema (table .getSchema ());
106120 return new LoggerSink (
107121 table .getSchema ().toRowDataType (),
108122 physicalSchema ,
109123 options .get (PRINT_IDENTIFIER ),
110- options .get (ALL_CHANGELOG_MODE ));
124+ options .get (RECORDS_PER_SECOND ),
125+ options .get (ALL_CHANGELOG_MODE ),
126+ options .get (MUTE_OUTPUT )
127+ );
111128 }
112129
113130 private static class LoggerSink implements DynamicTableSink {
@@ -116,12 +133,21 @@ private static class LoggerSink implements DynamicTableSink {
116133 private final TableSchema tableSchema ;
117134 private final boolean allChangeLogMode ;
118135 private final DataType type ;
136+ private final int recordsPerSecond ;
137+ private final boolean muteOutput ;
119138
120- public LoggerSink (DataType type , TableSchema tableSchema , String printIdentifier , boolean allChangeLogMode ) {
139+ public LoggerSink (DataType type ,
140+ TableSchema tableSchema ,
141+ String printIdentifier ,
142+ int recordsPerSecond ,
143+ boolean allChangeLogMode ,
144+ boolean muteOutput ) {
121145 this .type = type ;
122146 this .printIdentifier = printIdentifier ;
123147 this .tableSchema = tableSchema ;
148+ this .recordsPerSecond = recordsPerSecond ;
124149 this .allChangeLogMode = allChangeLogMode ;
150+ this .muteOutput = muteOutput ;
125151 }
126152
127153 @ Override
@@ -143,15 +169,16 @@ public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
143169 public SinkRuntimeProvider getSinkRuntimeProvider (Context context ) {
144170 DataStructureConverter converter = context .createDataStructureConverter (type );
145171 Slf4jSink .Builder <RowData > builder = Slf4jSink .<RowData >builder ()
146- .setFieldDataTypes (tableSchema .getFieldDataTypes ())
147172 .setPrintIdentifier (printIdentifier )
148- .setConverter (converter );
173+ .setRecordsPerSecond (recordsPerSecond )
174+ .setConverter (converter )
175+ .setMuteOutput (muteOutput );
149176 return SinkFunctionProvider .of (builder .build ());
150177 }
151178
152179 @ Override
153180 public DynamicTableSink copy () {
154- return new LoggerSink (type , tableSchema , printIdentifier , allChangeLogMode );
181+ return new LoggerSink (type , tableSchema , printIdentifier , recordsPerSecond , allChangeLogMode , muteOutput );
155182 }
156183
157184 @ Override
@@ -161,34 +188,50 @@ public String asSummaryString() {
161188 }
162189}
163190
164- class Slf4jSink <T > implements SinkFunction <T > {
191+ @ SuppressWarnings ("UnstableApiUsage" )
192+ class Slf4jSink <T > extends RichSinkFunction <T > {
165193 private static final long serialVersionUID = 1L ;
166194
167195 private static final Logger LOGGER = LoggerFactory .getLogger (Slf4jSink .class );
168196 private final String printIdentifier ;
169- private final RowData .FieldGetter [] fieldGetters ;
170- private static final String NULL_VALUE = "null" ;
171197 private final DynamicTableSink .DataStructureConverter converter ;
198+ private final int recordsPerSecond ;
199+ private final boolean muteOutput ;
200+ private transient RateLimiter rateLimiter ;
172201
173202 public Slf4jSink (String printIdentifier ,
174- LogicalType [] logicalTypes ,
175- DynamicTableSink .DataStructureConverter converter ) {
203+ DynamicTableSink .DataStructureConverter converter ,
204+ int recordsPerSecond ,
205+ boolean muteOutput ) {
206+
176207 this .printIdentifier = printIdentifier ;
177- this .fieldGetters = new RowData .FieldGetter [logicalTypes .length ];
178- for (int i = 0 ; i < logicalTypes .length ; i ++) {
179- fieldGetters [i ] = createFieldGetter (logicalTypes [i ], i );
180- }
181208 this .converter = converter ;
209+ this .recordsPerSecond = recordsPerSecond ;
210+ this .muteOutput = muteOutput ;
211+ }
212+
213+ @ Override
214+ public void open (Configuration parameters ) throws Exception {
215+ if (recordsPerSecond > 0 ) {
216+ LOGGER .info ("Sink output rate is limited to {} records per second." , recordsPerSecond );
217+ rateLimiter = RateLimiter .create (recordsPerSecond );
218+ }
219+
220+ if (muteOutput ) {
221+ LOGGER .info ("All records would be discarded because `{}` is set." , MUTE_OUTPUT .key ());
222+ }
182223 }
183224
184225 @ Override
185226 public void invoke (T value , Context context ) {
186- Object data = converter .toExternal (value );
187- StringBuilder builder = new StringBuilder ();
188- builder .append (printIdentifier );
189- builder .append ("-toString: " );
190- builder .append (data );
191- LOGGER .info (builder .toString ());
227+ if (rateLimiter != null ) {
228+ rateLimiter .acquire ();
229+ }
230+
231+ if (!muteOutput ) {
232+ Object data = converter .toExternal (value );
233+ LOGGER .info ("{}-toString: {}" , printIdentifier , data );
234+ }
192235 }
193236
194237 /**
@@ -205,17 +248,13 @@ public static <T> Builder<T> builder() {
205248 */
206249 public static class Builder <T > {
207250 private String printIdentifier ;
208- private DataType [] fieldDataTypes ;
209251 private DynamicTableSink .DataStructureConverter converter ;
252+ private int recordsPerSecond ;
253+ private boolean muteOutput ;
210254
211255 public Builder () {
212256 }
213257
214- public Builder <T > setFieldDataTypes (DataType [] fieldDataTypes ) {
215- this .fieldDataTypes = fieldDataTypes ;
216- return this ;
217- }
218-
219258 public Builder <T > setPrintIdentifier (String printIdentifier ) {
220259 this .printIdentifier = printIdentifier ;
221260 return this ;
@@ -226,12 +265,18 @@ public Builder<T> setConverter(DynamicTableSink.DataStructureConverter converter
226265 return this ;
227266 }
228267
268+ public Builder <T > setRecordsPerSecond (int recordsPerSecond ) {
269+ this .recordsPerSecond = recordsPerSecond ;
270+ return this ;
271+ }
272+
273+ public Builder <T > setMuteOutput (boolean muteOutput ) {
274+ this .muteOutput = muteOutput ;
275+ return this ;
276+ }
277+
229278 public Slf4jSink <T > build () {
230- final LogicalType [] logicalTypes =
231- Arrays .stream (fieldDataTypes )
232- .map (DataType ::getLogicalType )
233- .toArray (LogicalType []::new );
234- return new Slf4jSink <>(printIdentifier , logicalTypes , converter );
279+ return new Slf4jSink <>(printIdentifier , converter , recordsPerSecond , muteOutput );
235280 }
236281 }
237282}
0 commit comments