├── .idea
└── codeStyles
│ └── codeStyleConfig.xml
├── src
├── main
│ ├── java
│ │ └── org
│ │ │ └── kettle
│ │ │ └── beam
│ │ │ ├── pipeline
│ │ │ ├── BeamMetricsUpdatedListener.java
│ │ │ ├── handler
│ │ │ │ ├── BeamStepHandler.java
│ │ │ │ ├── BeamBigQueryInputStepHandler.java
│ │ │ │ ├── BeamTimestampStepHandler.java
│ │ │ │ ├── BeamInputStepHandler.java
│ │ │ │ ├── BeamKafkaOutputStepHandler.java
│ │ │ │ ├── BeamSubscriberStepHandler.java
│ │ │ │ ├── BeamGroupByStepHandler.java
│ │ │ │ ├── BeamPublisherStepHandler.java
│ │ │ │ ├── BeamBigQueryOutputStepHandler.java
│ │ │ │ ├── BeamBaseStepHandler.java
│ │ │ │ ├── BeamKafkaInputStepHandler.java
│ │ │ │ └── BeamOutputStepHandler.java
│ │ │ ├── spark
│ │ │ │ └── MainSpark.java
│ │ │ └── flink
│ │ │ │ └── MainFlink.java
│ │ │ ├── steps
│ │ │ ├── io
│ │ │ │ ├── BeamInputData.java
│ │ │ │ ├── BeamOutputData.java
│ │ │ │ ├── messages
│ │ │ │ │ └── messages_en_US.properties
│ │ │ │ ├── BeamOutput.java
│ │ │ │ ├── BeamInput.java
│ │ │ │ └── BeamInputMeta.java
│ │ │ ├── pubsub
│ │ │ │ ├── BeamPublishData.java
│ │ │ │ ├── BeamSubscribeData.java
│ │ │ │ ├── messages
│ │ │ │ │ └── messages_en_US.properties
│ │ │ │ ├── BeamPublish.java
│ │ │ │ ├── BeamSubscribe.java
│ │ │ │ └── BeamPublishMeta.java
│ │ │ ├── window
│ │ │ │ ├── messages
│ │ │ │ │ └── messages_en_US.properties
│ │ │ │ └── BeamTimestampMeta.java
│ │ │ ├── bq
│ │ │ │ ├── messages
│ │ │ │ │ └── messages_en_US.properties
│ │ │ │ └── BQField.java
│ │ │ └── kafka
│ │ │ │ ├── messages
│ │ │ │ └── messages_en_US.properties
│ │ │ │ ├── BeamConsume.java
│ │ │ │ ├── BeamProduce.java
│ │ │ │ ├── ConfigOption.java
│ │ │ │ └── BeamProduceMeta.java
│ │ │ ├── core
│ │ │ ├── partition
│ │ │ │ └── SinglePartitionFn.java
│ │ │ ├── shared
│ │ │ │ ├── AggregationType.java
│ │ │ │ ├── SimpleVariableSpace.java
│ │ │ │ └── VariableValue.java
│ │ │ ├── BeamDefaults.java
│ │ │ ├── Failure.java
│ │ │ ├── KettleRow.java
│ │ │ ├── util
│ │ │ │ ├── JsonRowMeta.java
│ │ │ │ └── KettleBeamUtil.java
│ │ │ ├── fn
│ │ │ │ ├── StringToKettleRowFn.java
│ │ │ │ ├── PublishStringsFn.java
│ │ │ │ ├── PubsubMessageToKettleRowFn.java
│ │ │ │ ├── KVLongStringToKettleRowFn.java
│ │ │ │ ├── KVStringStringToKettleRowFn.java
│ │ │ │ ├── KettleRowToKVStringStringFn.java
│ │ │ │ ├── PublishMessagesFn.java
│ │ │ │ ├── KettleToBQTableRowFn.java
│ │ │ │ ├── KettleToStringFn.java
│ │ │ │ ├── StringToKettleFn.java
│ │ │ │ ├── WindowInfoFn.java
│ │ │ │ ├── AssemblerFn.java
│ │ │ │ └── TimestampFn.java
│ │ │ └── transform
│ │ │ │ ├── BeamInputTransform.java
│ │ │ │ ├── BeamBQInputTransform.java
│ │ │ │ └── BeamKafkaOutputTransform.java
│ │ │ ├── xp
│ │ │ ├── BeamTrans.java
│ │ │ └── BeamDummyTrans.java
│ │ │ ├── metastore
│ │ │ ├── RunnerType.java
│ │ │ ├── JobParameter.java
│ │ │ ├── FieldDefinition.java
│ │ │ └── FileDefinition.java
│ │ │ ├── carte
│ │ │ ├── MetricsRegistrationQueue.java
│ │ │ ├── BeamMetricsEntry.java
│ │ │ └── RegisterBeamMetrics.java
│ │ │ └── perspective
│ │ │ ├── BeamMenuController.java
│ │ │ ├── TransDrawExtensionPoint.java
│ │ │ ├── BeamController.java
│ │ │ ├── messages
│ │ │ └── messages_en_US.properties
│ │ │ └── BeamSpoonPlugin.java
│ └── resources
│ │ ├── Trans-subclassing-idea
│ │ ├── beam_perspective.xul
│ │ ├── beam_transgraph_overlays.xul
│ │ ├── beam-kafka.svg
│ │ ├── beam_spoon_overlays.xul
│ │ ├── beam-bq-input.svg
│ │ └── beam-bq-output.svg
└── test
│ ├── java
│ ├── org
│ │ └── kettle
│ │ │ └── beam
│ │ │ └── transform
│ │ │ ├── BasePipelineTest.java
│ │ │ ├── GroupByPipelineTest.java
│ │ │ ├── MergeJoinPipelineTest.java
│ │ │ ├── SwitchCasePipelineTest.java
│ │ │ ├── StreamLookupPipelineTest.java
│ │ │ ├── FilterPipelineTest.java
│ │ │ └── PipelineTestBase.java
│ └── core
│ │ ├── coder
│ │ └── KettleRowCoderTest.java
│ │ ├── KettleRowTest.java
│ │ └── metastore
│ │ └── SerializableMetaStoreTest.java
│ └── resources
│ └── customers
│ └── state-data.txt
├── .gitignore
├── flink-notes-matt.txt
├── README.md
└── spark-notes-matt.txt
/.idea/codeStyles/codeStyleConfig.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 |
5 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/pipeline/BeamMetricsUpdatedListener.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.pipeline;
2 |
3 | import org.apache.beam.sdk.PipelineResult;
4 |
5 | public interface BeamMetricsUpdatedListener {
6 |
7 | void beamMetricsUpdated( PipelineResult pipelineResult );
8 | }
9 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/steps/io/BeamInputData.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.steps.io;
2 |
3 | import org.pentaho.di.trans.step.BaseStepData;
4 | import org.pentaho.di.trans.step.StepDataInterface;
5 |
6 | public class BeamInputData extends BaseStepData implements StepDataInterface {
7 |
8 | }
9 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/steps/io/BeamOutputData.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.steps.io;
2 |
3 | import org.pentaho.di.trans.step.BaseStepData;
4 | import org.pentaho.di.trans.step.StepDataInterface;
5 |
6 | public class BeamOutputData extends BaseStepData implements StepDataInterface {
7 |
8 | }
9 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/steps/pubsub/BeamPublishData.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.steps.pubsub;
2 |
3 | import org.pentaho.di.trans.step.BaseStepData;
4 | import org.pentaho.di.trans.step.StepDataInterface;
5 |
6 | public class BeamPublishData extends BaseStepData implements StepDataInterface {
7 |
8 | }
9 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/steps/pubsub/BeamSubscribeData.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.steps.pubsub;
2 |
3 | import org.pentaho.di.trans.step.BaseStepData;
4 | import org.pentaho.di.trans.step.StepDataInterface;
5 |
6 | public class BeamSubscribeData extends BaseStepData implements StepDataInterface {
7 |
8 | }
9 |
--------------------------------------------------------------------------------
/src/main/resources/Trans-subclassing-idea:
--------------------------------------------------------------------------------
1 |
2 |
3 | TransGraph : trans Usage
4 |
5 | isPaused()
6 | isInitializing()
7 | isRunning()
8 | isStopped()
9 | getErrors()
10 |
11 | killAll()
12 | stopAll()
13 |
14 |
15 |
16 | // Stub
17 | //
18 | addTransListener()
19 | trans.findRunThread()
20 | getLogChannelId()
21 | trans.setRepository()
22 | trans.setMetaStore()
23 | pauseRunning();
24 | resumeRunning();
25 |
26 |
--------------------------------------------------------------------------------
/.gitignore:
--------------------------------------------------------------------------------
1 | # Compiled class file
2 | *.class
3 |
4 | # Log file
5 | *.logChannel
6 |
7 | # BlueJ files
8 | *.ctxt
9 |
10 | # Mobile Tools for Java (J2ME)
11 | .mtj.tmp/
12 |
13 | # Package Files #
14 | *.jar
15 | *.war
16 | *.nar
17 | *.ear
18 | *.zip
19 | *.tar.gz
20 | *.rar
21 |
22 | # virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml
23 | hs_err_pid*
24 |
25 | .idea/
26 | target/
27 |
28 | *.kate-swp
29 |
30 | src/main/resources/*.png
31 |
32 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/core/partition/SinglePartitionFn.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.core.partition;
2 |
3 | import org.apache.beam.sdk.transforms.Partition;
4 | import org.kettle.beam.core.KettleRow;
5 |
6 | public class SinglePartitionFn implements Partition.PartitionFn {
7 |
8 | private static final long serialVersionUID = 95100000000000001L;
9 |
10 | @Override public int partitionFor( KettleRow elem, int numPartitions ) {
11 | return 0;
12 | }
13 | }
14 |
--------------------------------------------------------------------------------
/src/main/resources/beam_perspective.xul:
--------------------------------------------------------------------------------
1 |
2 |
5 |
6 |
7 |
8 |
9 |
10 |
11 |
12 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/steps/io/messages/messages_en_US.properties:
--------------------------------------------------------------------------------
1 | BeamInputDialog.DialogTitle = Beam Input
2 | BeamInputDialog.InputLocation = Input location
3 | BeamInputDialog.FileDefinition = File definition to use
4 |
5 | BeamOutputDialog.DialogTitle = Beam Output
6 | BeamOutputDialog.OutputLocation = Output location
7 | BeamOutputDialog.FilePrefix = File prefix
8 | BeamOutputDialog.Windowed = Windowed writes?
9 | BeamOutputDialog.FileDefinition = File definition to use
10 | BeamOutputDialog.FileSuffix = File suffix
11 |
--------------------------------------------------------------------------------
/src/test/java/org/kettle/beam/transform/BasePipelineTest.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.transform;
2 |
3 | import org.junit.Test;
4 | import org.kettle.beam.util.BeamTransMetaUtil;
5 | import org.pentaho.di.trans.TransMeta;
6 |
7 | public class BasePipelineTest extends PipelineTestBase {
8 |
9 | @Test
10 | public void testBasicPipeline() throws Exception {
11 |
12 | TransMeta transMeta = BeamTransMetaUtil.generateBeamInputOutputTransMeta(
13 | "io-dummy-output",
14 | "INPUT",
15 | "OUTPUT",
16 | metaStore
17 | );
18 |
19 | createRunPipeline( transMeta );
20 | }
21 | }
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/steps/pubsub/messages/messages_en_US.properties:
--------------------------------------------------------------------------------
1 | BeamPublishDialog.DialogTitle = Beam Publish
2 | BeamPublishDialog.Topic = Topic to publish to
3 | BeamPublishDialog.MessageType = The type of message to publish
4 | BeamPublishDialog.MessageField = The field to use as message
5 |
6 | BeamSubscribeDialog.DialogTitle = Beam Subscribe
7 | BeamSubscribeDialog.Subscription = Subscription name
8 | BeamSubscribeDialog.Topic = Topic to subscribe to
9 | BeamSubscribeDialog.MessageType = The type of message to read
10 | BeamSubscribeDialog.MessageField = The output field name
11 |
12 |
13 |
14 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/core/shared/AggregationType.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.core.shared;
2 |
3 | import org.pentaho.di.core.exception.KettleException;
4 |
5 | public enum AggregationType {
6 | SUM, AVERAGE, COUNT_ALL, MIN, MAX, FIRST_INCL_NULL, LAST_INCL_NULL, FIRST, LAST,
7 | ;
8 |
9 | public static final AggregationType getTypeFromName( String name) throws KettleException {
10 | for ( AggregationType type : values()) {
11 | if (name.equals( type.name() )) {
12 | return type;
13 | }
14 | }
15 | throw new KettleException( "Aggregation type '"+name+"' is not recognized or supported" );
16 | }
17 | }
18 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/steps/window/messages/messages_en_US.properties:
--------------------------------------------------------------------------------
1 | BeamWindowDialog.DialogTitle = Beam Window
2 | BeamWindowDialog.WindowType = Window type
3 | BeamWindowDialog.Duration = Window size (duration in seconds)
4 | BeamWindowDialog.Every = Every x seconds (Sliding windows)
5 | BeamWindowDialog.StartTimeField = Window start field
6 | BeamWindowDialog.EndTimeField = Window end field
7 | BeamWindowDialog.MaxTimeField = Window max field
8 |
9 | BeamTimestampDialog.DialogTitle = Beam Timestamp
10 | BeamTimestampDialog.FieldName = The time field to use. Uses system time if blank
11 | BeamTimestampDialog.Reading = Get timestamp from stream?
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/xp/BeamTrans.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.xp;
2 |
3 | import org.pentaho.di.trans.Trans;
4 | import org.pentaho.di.trans.TransMeta;
5 | import org.pentaho.di.trans.step.StepDataInterface;
6 |
7 | public class BeamTrans extends Trans {
8 |
9 | public BeamTrans( TransMeta transMeta ) {
10 | super(transMeta);
11 | }
12 |
13 | public void setFinished( boolean finished) {
14 | super.setFinished( finished );
15 | }
16 |
17 | // Always only 1 step copy: 0
18 | //
19 | public StepDataInterface getStepDataInterface( String stepname, int stepcopy) {
20 | return super.getStepDataInterface( stepname, 0 );
21 | }
22 | }
23 |
--------------------------------------------------------------------------------
/src/test/java/org/kettle/beam/transform/GroupByPipelineTest.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.transform;
2 |
3 | import org.junit.Test;
4 | import org.kettle.beam.util.BeamTransMetaUtil;
5 | import org.pentaho.di.trans.TransMeta;
6 |
7 | public class GroupByPipelineTest extends PipelineTestBase {
8 |
9 | @Test
10 | public void testGroupByPipeline() throws Exception {
11 |
12 | TransMeta transMeta = BeamTransMetaUtil.generateBeamGroupByTransMeta(
13 | "io-group-output",
14 | "INPUT",
15 | "OUTPUT",
16 | metaStore
17 | );
18 |
19 | try {
20 | createRunPipeline( transMeta );
21 | } catch(Exception e) {
22 | e.printStackTrace();
23 | throw e;
24 | }
25 | }
26 | }
--------------------------------------------------------------------------------
/src/test/java/org/kettle/beam/transform/MergeJoinPipelineTest.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.transform;
2 |
3 | import org.junit.Test;
4 | import org.kettle.beam.util.BeamTransMetaUtil;
5 | import org.pentaho.di.trans.TransMeta;
6 |
7 | public class MergeJoinPipelineTest extends PipelineTestBase {
8 |
9 | @Test
10 | public void testMergeJoinPipeline() throws Exception {
11 |
12 | TransMeta transMeta = BeamTransMetaUtil.generateMergeJoinTransMeta(
13 | "inputs-merge-join-output",
14 | "INPUT",
15 | "OUTPUT",
16 | metaStore
17 | );
18 |
19 | try {
20 | createRunPipeline( transMeta );
21 | } catch(Exception e) {
22 | e.printStackTrace();
23 | throw e;
24 | }
25 | }
26 |
27 | }
--------------------------------------------------------------------------------
/src/test/java/org/kettle/beam/transform/SwitchCasePipelineTest.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.transform;
2 |
3 | import org.junit.Test;
4 | import org.kettle.beam.util.BeamTransMetaUtil;
5 | import org.pentaho.di.trans.TransMeta;
6 |
7 | public class SwitchCasePipelineTest extends PipelineTestBase {
8 |
9 | @Test
10 | public void testSwitchCasePipeline() throws Exception {
11 |
12 | TransMeta transMeta = BeamTransMetaUtil.generateSwitchCaseTransMeta(
13 | "io-switch-case-output",
14 | "INPUT",
15 | "OUTPUT",
16 | metaStore
17 | );
18 |
19 | try {
20 | createRunPipeline( transMeta );
21 | } catch(Exception e) {
22 | e.printStackTrace();
23 | throw e;
24 | }
25 | }
26 |
27 | }
--------------------------------------------------------------------------------
/src/test/java/org/kettle/beam/transform/StreamLookupPipelineTest.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.transform;
2 |
3 | import org.junit.Test;
4 | import org.kettle.beam.util.BeamTransMetaUtil;
5 | import org.pentaho.di.trans.TransMeta;
6 |
7 | public class StreamLookupPipelineTest extends PipelineTestBase {
8 |
9 | @Test
10 | public void testStreamLookupPipeline() throws Exception {
11 |
12 | TransMeta transMeta = BeamTransMetaUtil.generateStreamLookupTransMeta(
13 | "io-stream-lookup-output",
14 | "INPUT",
15 | "OUTPUT",
16 | metaStore
17 | );
18 |
19 | try {
20 | createRunPipeline( transMeta );
21 | } catch(Exception e) {
22 | e.printStackTrace();
23 | throw e;
24 | }
25 | }
26 |
27 | }
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/core/shared/SimpleVariableSpace.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.core.shared;
2 |
3 | import java.io.Serializable;
4 | import java.util.ArrayList;
5 | import java.util.List;
6 |
7 | /**
8 | * A variablespace which simply is easy to serialize.
9 | */
10 | public class SimpleVariableSpace implements Serializable {
11 |
12 | private List variables;
13 |
14 | public SimpleVariableSpace() {
15 | variables = new ArrayList<>();
16 | }
17 |
18 | /**
19 | * Gets variables
20 | *
21 | * @return value of variables
22 | */
23 | public List getVariables() {
24 | return variables;
25 | }
26 |
27 | /**
28 | * @param variables The variables to set
29 | */
30 | public void setVariables( List variables ) {
31 | this.variables = variables;
32 | }
33 | }
34 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/xp/BeamDummyTrans.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.xp;
2 |
3 | import org.pentaho.di.trans.Trans;
4 | import org.pentaho.di.trans.TransMeta;
5 | import org.pentaho.di.trans.step.StepDataInterface;
6 | import org.pentaho.di.trans.step.StepInterface;
7 | import org.pentaho.di.trans.step.StepMeta;
8 | import org.pentaho.di.trans.steps.dummytrans.DummyTrans;
9 |
10 | public class BeamDummyTrans extends DummyTrans implements StepInterface {
11 |
12 | protected boolean init;
13 |
14 | public BeamDummyTrans( StepMeta stepMeta, StepDataInterface stepDataInterface, int copyNr, TransMeta transMeta, Trans trans ) {
15 | super( stepMeta, stepDataInterface, copyNr, transMeta, trans );
16 | }
17 |
18 | public void setInit( boolean init ) {
19 | this.init = init;
20 | }
21 |
22 | @Override public boolean isInitialising() {
23 | return init;
24 | }
25 | }
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/metastore/RunnerType.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.metastore;
2 |
3 | public enum RunnerType {
4 | Direct,
5 | DataFlow,
6 | Spark,
7 | Flink,
8 | ;
9 |
10 | public static String[] getNames() {
11 | String[] names = new String[values().length];
12 | for (int i=0;i> stepCollectionMap,
19 | Pipeline pipeline,
20 | RowMetaInterface rowMeta,
21 | List previousSteps,
22 | PCollection input
23 | ) throws KettleException;
24 |
25 | boolean isInput();
26 |
27 | boolean isOutput();
28 | }
29 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/core/shared/VariableValue.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.core.shared;
2 |
3 | import java.io.Serializable;
4 |
5 | public class VariableValue implements Serializable {
6 |
7 | private String variable;
8 | private String value;
9 |
10 | public VariableValue() {
11 | }
12 |
13 | public VariableValue( String variable, String value ) {
14 | this.variable = variable;
15 | this.value = value;
16 | }
17 |
18 | /**
19 | * Gets variable
20 | *
21 | * @return value of variable
22 | */
23 | public String getVariable() {
24 | return variable;
25 | }
26 |
27 | /**
28 | * @param variable The variable to set
29 | */
30 | public void setVariable( String variable ) {
31 | this.variable = variable;
32 | }
33 |
34 | /**
35 | * Gets value
36 | *
37 | * @return value of value
38 | */
39 | public String getValue() {
40 | return value;
41 | }
42 |
43 | /**
44 | * @param value The value to set
45 | */
46 | public void setValue( String value ) {
47 | this.value = value;
48 | }
49 | }
50 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/carte/MetricsRegistrationQueue.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.carte;
2 |
3 | import java.util.Queue;
4 | import java.util.concurrent.ConcurrentLinkedQueue;
5 |
6 | import org.pentaho.di.core.logging.LogChannel;
7 |
8 | public class MetricsRegistrationQueue {
9 | private static MetricsRegistrationQueue registrationQueue;
10 |
11 | private Queue entryQueue;
12 |
13 | public synchronized static MetricsRegistrationQueue getInstance() {
14 | if (registrationQueue==null) {
15 | registrationQueue = new MetricsRegistrationQueue();
16 | }
17 | return registrationQueue;
18 | }
19 |
20 | private MetricsRegistrationQueue() {
21 | entryQueue = new ConcurrentLinkedQueue();
22 | }
23 |
24 | public void addNodeRegistryEntry( BeamMetricsEntry entry) {
25 | entryQueue.add(entry);
26 | LogChannel.GENERAL.logDetailed("Beam Metrics update received for job id : "+entry.getCarteObjectId()+", trans="+entry.getTrans());
27 | }
28 |
29 | public BeamMetricsEntry pollEntry() {
30 | return entryQueue.poll();
31 | }
32 | }
33 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/core/BeamDefaults.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.core;
2 |
3 | public class BeamDefaults {
4 | public static final String PUBSUB_MESSAGE_TYPE_AVROS = "Avros";
5 | public static final String PUBSUB_MESSAGE_TYPE_PROTOBUF = "protobuf";
6 | public static final String PUBSUB_MESSAGE_TYPE_STRING = "String";
7 | public static final String PUBSUB_MESSAGE_TYPE_MESSAGE = "PubsubMessage";
8 |
9 | public static final String[] PUBSUB_MESSAGE_TYPES = new String[] {
10 | // PUBSUB_MESSAGE_TYPE_AVROS,
11 | // PUBSUB_MESSAGE_TYPE_PROTOBUF,
12 | PUBSUB_MESSAGE_TYPE_STRING,
13 | PUBSUB_MESSAGE_TYPE_MESSAGE,
14 | };
15 |
16 |
17 | public static final String WINDOW_TYPE_FIXED = "Fixed";
18 | public static final String WINDOW_TYPE_SLIDING = "Sliding";
19 | public static final String WINDOW_TYPE_SESSION = "Session";
20 | public static final String WINDOW_TYPE_GLOBAL = "Global";
21 |
22 | public static final String[] WINDOW_TYPES = new String[] {
23 | WINDOW_TYPE_FIXED,
24 | WINDOW_TYPE_SLIDING,
25 | WINDOW_TYPE_SESSION,
26 | WINDOW_TYPE_GLOBAL
27 | };
28 |
29 | }
30 |
--------------------------------------------------------------------------------
/src/main/resources/beam_transgraph_overlays.xul:
--------------------------------------------------------------------------------
1 |
2 |
3 |
27 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/metastore/JobParameter.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.metastore;
2 |
3 | import org.pentaho.metastore.persist.MetaStoreAttribute;
4 |
5 | public class JobParameter {
6 |
7 | @MetaStoreAttribute
8 | private String variable;
9 |
10 | @MetaStoreAttribute
11 | private String value;
12 |
13 | public JobParameter() {
14 | }
15 |
16 | public JobParameter( String variable, String value ) {
17 | this.variable = variable;
18 | this.value = value;
19 | }
20 |
21 | /**
22 | * Gets variable
23 | *
24 | * @return value of variable
25 | */
26 | public String getVariable() {
27 | return variable;
28 | }
29 |
30 | /**
31 | * @param variable The variable to set
32 | */
33 | public void setVariable( String variable ) {
34 | this.variable = variable;
35 | }
36 |
37 | /**
38 | * Gets value
39 | *
40 | * @return value of value
41 | */
42 | public String getValue() {
43 | return value;
44 | }
45 |
46 | /**
47 | * @param value The value to set
48 | */
49 | public void setValue( String value ) {
50 | this.value = value;
51 | }
52 | }
53 |
--------------------------------------------------------------------------------
/src/test/resources/customers/state-data.txt:
--------------------------------------------------------------------------------
1 | Alabama;4874747
2 | ALASKA;739795
3 | Arizona;7016270
4 | Arkansas;3004279
5 | CALIFORNIA;39536653
6 | Colorado;5607154
7 | Connecticut;3588184
8 | Delaware;961939
9 | District of Columbia;693972
10 | FLORIDA;20984400
11 | Georgia;10429379
12 | Hawaii;1427538
13 | Idaho;1716943
14 | Illinois;12802023
15 | INDIANA;6666818
16 | Iowa;3145711
17 | Kansas;2913123
18 | Kentucky;4454189
19 | Louisiana;4684333
20 | Maine;1335907
21 | Maryland;6052177
22 | Massachusetts;6859819
23 | Michigan;9962311
24 | Minnesota;5576606
25 | Mississippi;2984100
26 | Missouri;6113532
27 | Montana;1050493
28 | NEBRASKA;1920076
29 | Nevada;2998039
30 | New Hampshire;1342795
31 | New Jersey;9005644
32 | New Mexico;2088070
33 | NEW YORK;19849399
34 | North Carolina;10273419
35 | North Dakota;755393
36 | Ohio;11658609
37 | Oklahoma;3930864
38 | Oregon;4142776
39 | Pennsylvania;12805537
40 | Rhode Island;1059639
41 | South Carolina;5024369
42 | South Dakota;869666
43 | Tennessee;6715984
44 | TEXAS;28304596
45 | Utah;3101833
46 | Vermont;623657
47 | Virginia;8470020
48 | WASHINGTON;7405743
49 | West Virginia;1815857
50 | Wisconsin;5795483
51 | Wyoming;579315
52 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/steps/kafka/messages/messages_en_US.properties:
--------------------------------------------------------------------------------
1 | BeamProduceDialog.DialogTitle = Beam Kafka Produce
2 | BeamProduceDialog.BootstrapServers = Bootstrap servers
3 | BeamProduceDialog.Topic = The topics
4 | BeamConsumeDialog.GroupId = Group ID
5 | BeamProduceDialog.KeyField = The field to use as key
6 | BeamProduceDialog.MessageField = The field to use as message
7 |
8 | BeamConsumeDialog.DialogTitle = Beam Kafka Consume
9 | BeamConsumeDialog.BootstrapServers = Bootstrap servers
10 | BeamConsumeDialog.Topics = Topics to consume
11 | BeamConsumeDialog.KeyField = The name of the key output field
12 | BeamConsumeDialog.MessageField = The name of the message output field
13 |
14 | BeamProduceDialog.UseProcessingTime = Use processing time
15 | BeamProduceDialog.UseLogAppendTime = Use log append time
16 | BeamProduceDialog.UseCreateTime = Use create time
17 | BeamProduceDialog.RestrictToCommitted = Restrict read to committed messages
18 | BeamProduceDialog.AllowCommitConsumed = Allow offsets to be committed back
19 | BeamProduceDialog.ConfigOptions = Configuration options
20 | BeamProduceDialog.ConfigOptions.Column.Parameter = Parameter
21 | BeamProduceDialog.ConfigOptions.Column.Value = Value
22 | BeamProduceDialog.ConfigOptions.Column.Type = Type
23 |
24 |
--------------------------------------------------------------------------------
/src/test/java/core/coder/KettleRowCoderTest.java:
--------------------------------------------------------------------------------
1 | package core.coder;
2 |
3 | import junit.framework.TestCase;
4 | import org.junit.Test;
5 | import org.kettle.beam.core.KettleRow;
6 | import org.kettle.beam.core.coder.KettleRowCoder;
7 |
8 | import java.io.ByteArrayInputStream;
9 | import java.io.ByteArrayOutputStream;
10 | import java.io.IOException;
11 | import java.util.Date;
12 |
13 | public class KettleRowCoderTest extends TestCase {
14 |
15 | ByteArrayOutputStream outputStream;
16 | private KettleRowCoder kettleRowCoder;
17 |
18 | @Override protected void setUp() throws Exception {
19 |
20 | outputStream= new ByteArrayOutputStream( 1000000 );
21 | kettleRowCoder = new KettleRowCoder();
22 | }
23 |
24 | @Test
25 | public void testEncode() throws IOException {
26 |
27 | KettleRow row1 = new KettleRow(new Object[] { "AAA", "BBB", Long.valueOf( 100 ), Double.valueOf(1.234), new Date( 876876868 ) } );
28 |
29 | kettleRowCoder.encode( row1, outputStream );
30 | outputStream.flush();
31 | outputStream.close();
32 | byte[] bytes = outputStream.toByteArray();
33 |
34 | ByteArrayInputStream inputStream = new ByteArrayInputStream( bytes );
35 | KettleRow row1d = kettleRowCoder.decode( inputStream );
36 |
37 | assertEquals( row1, row1d );
38 | }
39 |
40 |
41 | @Test
42 | public void decode() {
43 | }
44 | }
--------------------------------------------------------------------------------
/src/main/resources/beam-kafka.svg:
--------------------------------------------------------------------------------
1 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/perspective/BeamMenuController.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2017 Hitachi America, Ltd., R&D.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.kettle.beam.perspective;
18 |
19 | import org.pentaho.di.ui.spoon.ISpoonMenuController;
20 | import org.pentaho.ui.xul.dom.Document;
21 | import org.pentaho.ui.xul.impl.AbstractXulEventHandler;
22 |
23 | public class BeamMenuController extends AbstractXulEventHandler implements ISpoonMenuController {
24 |
25 | private static final Class> PKG = BeamController.class;
26 |
27 | private BeamController beamController;
28 |
29 | public void setBeamController( BeamController beamController ) {
30 | this.beamController = beamController;
31 | }
32 |
33 |
34 | @Override public void updateMenu( Document document ) {
35 | // TODO
36 | }
37 | }
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/pipeline/spark/MainSpark.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.pipeline.spark;
2 |
3 | import org.apache.commons.io.IOUtils;
4 | import org.apache.hadoop.conf.Configuration;
5 | import org.apache.hadoop.fs.FSDataInputStream;
6 | import org.apache.hadoop.fs.FileSystem;
7 | import org.apache.hadoop.fs.Path;
8 | import org.kettle.beam.core.BeamKettle;
9 | import org.kettle.beam.core.metastore.SerializableMetaStore;
10 | import org.kettle.beam.metastore.BeamJobConfig;
11 | import org.kettle.beam.pipeline.KettleBeamPipelineExecutor;
12 | import org.kettle.beam.pipeline.main.MainBeam;
13 | import org.kettle.beam.util.BeamConst;
14 | import org.pentaho.di.core.exception.KettleException;
15 | import org.pentaho.di.core.logging.LogChannel;
16 | import org.pentaho.di.core.plugins.PluginInterface;
17 | import org.pentaho.di.core.plugins.PluginRegistry;
18 | import org.pentaho.di.core.plugins.StepPluginType;
19 | import org.pentaho.di.core.xml.XMLHandler;
20 | import org.pentaho.di.trans.TransMeta;
21 | import org.pentaho.metastore.api.IMetaStore;
22 | import org.pentaho.metastore.persist.MetaStoreFactory;
23 | import org.pentaho.metastore.util.PentahoDefaults;
24 |
25 | import java.io.IOException;
26 | import java.util.ArrayList;
27 | import java.util.Arrays;
28 | import java.util.List;
29 |
30 | public class MainSpark {
31 | public static void main( String[] args ) {
32 | MainBeam.mainMethod(args, "Apache Spark");
33 | }
34 | }
35 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/pipeline/flink/MainFlink.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.pipeline.flink;
2 |
3 | import org.apache.commons.io.IOUtils;
4 | import org.apache.hadoop.conf.Configuration;
5 | import org.apache.hadoop.fs.FSDataInputStream;
6 | import org.apache.hadoop.fs.FileSystem;
7 | import org.apache.hadoop.fs.Path;
8 | import org.kettle.beam.core.BeamKettle;
9 | import org.kettle.beam.core.metastore.SerializableMetaStore;
10 | import org.kettle.beam.metastore.BeamJobConfig;
11 | import org.kettle.beam.pipeline.KettleBeamPipelineExecutor;
12 | import org.kettle.beam.pipeline.main.MainBeam;
13 | import org.kettle.beam.util.BeamConst;
14 | import org.pentaho.di.core.Const;
15 | import org.pentaho.di.core.exception.KettleException;
16 | import org.pentaho.di.core.logging.LogChannel;
17 | import org.pentaho.di.core.plugins.PluginInterface;
18 | import org.pentaho.di.core.plugins.PluginRegistry;
19 | import org.pentaho.di.core.plugins.StepPluginType;
20 | import org.pentaho.di.core.xml.XMLHandler;
21 | import org.pentaho.di.trans.TransMeta;
22 | import org.pentaho.metastore.api.IMetaStore;
23 | import org.pentaho.metastore.persist.MetaStoreFactory;
24 | import org.pentaho.metastore.util.PentahoDefaults;
25 |
26 | import java.io.IOException;
27 | import java.util.ArrayList;
28 | import java.util.Arrays;
29 | import java.util.List;
30 |
31 | public class MainFlink {
32 | public static void main( String[] args ) {
33 | MainBeam.mainMethod(args, "Apache Flink");
34 | }
35 | }
36 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/core/Failure.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.core;
2 |
3 | import java.io.Serializable;
4 |
5 | public class Failure implements Serializable {
6 |
7 | private String failingClass;
8 | private String stackTrace;
9 | private String sourceData;
10 |
11 | public Failure() {
12 | }
13 |
14 | public Failure( String failingClass, String stackTrace, String sourceData ) {
15 | this.failingClass = failingClass;
16 | this.stackTrace = stackTrace;
17 | this.sourceData = sourceData;
18 | }
19 |
20 | /**
21 | * Gets failingClass
22 | *
23 | * @return value of failingClass
24 | */
25 | public String getFailingClass() {
26 | return failingClass;
27 | }
28 |
29 | /**
30 | * @param failingClass The failingClass to set
31 | */
32 | public void setFailingClass( String failingClass ) {
33 | this.failingClass = failingClass;
34 | }
35 |
36 | /**
37 | * Gets stackTrace
38 | *
39 | * @return value of stackTrace
40 | */
41 | public String getStackTrace() {
42 | return stackTrace;
43 | }
44 |
45 | /**
46 | * @param stackTrace The stackTrace to set
47 | */
48 | public void setStackTrace( String stackTrace ) {
49 | this.stackTrace = stackTrace;
50 | }
51 |
52 | /**
53 | * Gets sourceData
54 | *
55 | * @return value of sourceData
56 | */
57 | public String getSourceData() {
58 | return sourceData;
59 | }
60 |
61 | /**
62 | * @param sourceData The sourceData to set
63 | */
64 | public void setSourceData( String sourceData ) {
65 | this.sourceData = sourceData;
66 | }
67 | }
68 |
--------------------------------------------------------------------------------
/src/test/java/org/kettle/beam/transform/FilterPipelineTest.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.transform;
2 |
3 | import junit.framework.TestCase;
4 | import org.apache.beam.runners.direct.DirectRunner;
5 | import org.apache.beam.sdk.Pipeline;
6 | import org.apache.beam.sdk.PipelineResult;
7 | import org.apache.beam.sdk.metrics.MetricQueryResults;
8 | import org.apache.beam.sdk.metrics.MetricResult;
9 | import org.apache.beam.sdk.metrics.MetricResults;
10 | import org.apache.beam.sdk.metrics.MetricsFilter;
11 | import org.apache.beam.sdk.options.PipelineOptions;
12 | import org.apache.beam.sdk.options.PipelineOptionsFactory;
13 | import org.apache.commons.io.FileUtils;
14 | import org.junit.Ignore;
15 | import org.junit.Test;
16 | import org.kettle.beam.core.BeamDefaults;
17 | import org.kettle.beam.core.BeamKettle;
18 | import org.kettle.beam.pipeline.TransMetaPipelineConverter;
19 | import org.kettle.beam.util.BeamTransMetaUtil;
20 | import org.pentaho.di.trans.TransMeta;
21 | import org.pentaho.metastore.api.IMetaStore;
22 | import org.pentaho.metastore.stores.memory.MemoryMetaStore;
23 |
24 | import java.io.File;
25 |
26 | public class FilterPipelineTest extends PipelineTestBase {
27 |
28 | @Test
29 | public void testFilterRowsPipeline() throws Exception {
30 |
31 | TransMeta transMeta = BeamTransMetaUtil.generateFilterRowsTransMeta(
32 | "io-filter-rows-output",
33 | "INPUT",
34 | "OUTPUT",
35 | metaStore
36 | );
37 |
38 | try {
39 | createRunPipeline( transMeta );
40 | } catch(Exception e) {
41 | e.printStackTrace();
42 | throw e;
43 | }
44 | }
45 |
46 | }
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/steps/bq/BQField.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.steps.bq;
2 |
3 | import org.apache.commons.lang.StringUtils;
4 |
5 | public class BQField {
6 | private String name;
7 | private String newName;
8 | private String kettleType;
9 |
10 | public BQField() {
11 | }
12 |
13 | public BQField( String name, String newName, String kettleType ) {
14 | this.name = name;
15 | this.newName = newName;
16 | this.kettleType = kettleType;
17 | }
18 |
19 | public String getNewNameOrName() {
20 | if ( StringUtils.isNotEmpty(newName)) {
21 | return newName;
22 | } else {
23 | return name;
24 | }
25 | }
26 |
27 | /**
28 | * Gets name
29 | *
30 | * @return value of name
31 | */
32 | public String getName() {
33 | return name;
34 | }
35 |
36 | /**
37 | * @param name The name to set
38 | */
39 | public void setName( String name ) {
40 | this.name = name;
41 | }
42 |
43 | /**
44 | * Gets newName
45 | *
46 | * @return value of newName
47 | */
48 | public String getNewName() {
49 | return newName;
50 | }
51 |
52 | /**
53 | * @param newName The newName to set
54 | */
55 | public void setNewName( String newName ) {
56 | this.newName = newName;
57 | }
58 |
59 | /**
60 | * Gets kettleType
61 | *
62 | * @return value of kettleType
63 | */
64 | public String getKettleType() {
65 | return kettleType;
66 | }
67 |
68 | /**
69 | * @param kettleType The kettleType to set
70 | */
71 | public void setKettleType( String kettleType ) {
72 | this.kettleType = kettleType;
73 | }
74 | }
75 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/steps/io/BeamOutput.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.steps.io;
2 |
3 | import org.pentaho.di.core.exception.KettleException;
4 | import org.pentaho.di.trans.Trans;
5 | import org.pentaho.di.trans.TransMeta;
6 | import org.pentaho.di.trans.step.BaseStep;
7 | import org.pentaho.di.trans.step.StepDataInterface;
8 | import org.pentaho.di.trans.step.StepInterface;
9 | import org.pentaho.di.trans.step.StepMeta;
10 | import org.pentaho.di.trans.step.StepMetaInterface;
11 |
12 | public class BeamOutput extends BaseStep implements StepInterface {
13 |
14 | /**
15 | * This is the base step that forms that basis for all steps. You can derive from this class to implement your own
16 | * steps.
17 | *
18 | * @param stepMeta The StepMeta object to run.
19 | * @param stepDataInterface the data object to store temporary data, database connections, caches, result sets,
20 | * hashtables etc.
21 | * @param copyNr The copynumber for this step.
22 | * @param transMeta The TransInfo of which the step stepMeta is part of.
23 | * @param trans The (running) transformation to obtain information shared among the steps.
24 | */
25 | public BeamOutput( StepMeta stepMeta, StepDataInterface stepDataInterface, int copyNr, TransMeta transMeta,
26 | Trans trans ) {
27 | super( stepMeta, stepDataInterface, copyNr, transMeta, trans );
28 | }
29 |
30 | @Override public boolean processRow( StepMetaInterface smi, StepDataInterface sdi ) throws KettleException {
31 | Object[] row = getRow();
32 | if (row==null) {
33 | setOutputDone();
34 | return false;
35 | }
36 | putRow(getInputRowMeta(), row);
37 | return true;
38 | }
39 | }
40 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/steps/io/BeamInput.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.steps.io;
2 |
3 | import org.pentaho.di.core.exception.KettleException;
4 | import org.pentaho.di.trans.Trans;
5 | import org.pentaho.di.trans.TransMeta;
6 | import org.pentaho.di.trans.step.BaseStep;
7 | import org.pentaho.di.trans.step.StepDataInterface;
8 | import org.pentaho.di.trans.step.StepInterface;
9 | import org.pentaho.di.trans.step.StepMeta;
10 | import org.pentaho.di.trans.step.StepMetaInterface;
11 |
12 | public class BeamInput extends BaseStep implements StepInterface {
13 |
14 | /**
15 | * This is the base step that forms that basis for all steps. You can derive from this class to implement your own
16 | * steps.
17 | *
18 | * @param stepMeta The StepMeta object to run.
19 | * @param stepDataInterface the data object to store temporary data, database connections, caches, result sets,
20 | * hashtables etc.
21 | * @param copyNr The copynumber for this step.
22 | * @param transMeta The TransInfo of which the step stepMeta is part of.
23 | * @param trans The (running) transformation to obtain information shared among the steps.
24 | */
25 | public BeamInput( StepMeta stepMeta, StepDataInterface stepDataInterface, int copyNr, TransMeta transMeta,
26 | Trans trans ) {
27 | super( stepMeta, stepDataInterface, copyNr, transMeta, trans );
28 | }
29 |
30 | @Override public boolean processRow( StepMetaInterface smi, StepDataInterface sdi ) throws KettleException {
31 |
32 |
33 | // Outside of Beam this step doesn't actually do anything, it's just metadata
34 | // This step gets converted into Beam API calls in a pipeline
35 | //
36 | return false;
37 | }
38 | }
39 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/steps/kafka/BeamConsume.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.steps.kafka;
2 |
3 | import org.pentaho.di.core.exception.KettleException;
4 | import org.pentaho.di.trans.Trans;
5 | import org.pentaho.di.trans.TransMeta;
6 | import org.pentaho.di.trans.step.BaseStep;
7 | import org.pentaho.di.trans.step.StepDataInterface;
8 | import org.pentaho.di.trans.step.StepInterface;
9 | import org.pentaho.di.trans.step.StepMeta;
10 | import org.pentaho.di.trans.step.StepMetaInterface;
11 |
12 | public class BeamConsume extends BaseStep implements StepInterface {
13 |
14 | /**
15 | * This is the base step that forms that basis for all steps. You can derive from this class to implement your own
16 | * steps.
17 | *
18 | * @param stepMeta The StepMeta object to run.
19 | * @param stepDataInterface the data object to store temporary data, database connections, caches, result sets,
20 | * hashtables etc.
21 | * @param copyNr The copynumber for this step.
22 | * @param transMeta The TransInfo of which the step stepMeta is part of.
23 | * @param trans The (running) transformation to obtain information shared among the steps.
24 | */
25 | public BeamConsume( StepMeta stepMeta, StepDataInterface stepDataInterface, int copyNr, TransMeta transMeta,
26 | Trans trans ) {
27 | super( stepMeta, stepDataInterface, copyNr, transMeta, trans );
28 | }
29 |
30 | @Override public boolean processRow( StepMetaInterface smi, StepDataInterface sdi ) throws KettleException {
31 |
32 | // Outside of a Beam Runner this step doesn't actually do anything, it's just metadata
33 | // This step gets converted into Beam API calls in a pipeline
34 | //
35 | return false;
36 | }
37 | }
38 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/steps/kafka/BeamProduce.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.steps.kafka;
2 |
3 | import org.pentaho.di.core.exception.KettleException;
4 | import org.pentaho.di.trans.Trans;
5 | import org.pentaho.di.trans.TransMeta;
6 | import org.pentaho.di.trans.step.BaseStep;
7 | import org.pentaho.di.trans.step.StepDataInterface;
8 | import org.pentaho.di.trans.step.StepInterface;
9 | import org.pentaho.di.trans.step.StepMeta;
10 | import org.pentaho.di.trans.step.StepMetaInterface;
11 |
12 | public class BeamProduce extends BaseStep implements StepInterface {
13 |
14 | /**
15 | * This is the base step that forms that basis for all steps. You can derive from this class to implement your own
16 | * steps.
17 | *
18 | * @param stepMeta The StepMeta object to run.
19 | * @param stepDataInterface the data object to store temporary data, database connections, caches, result sets,
20 | * hashtables etc.
21 | * @param copyNr The copynumber for this step.
22 | * @param transMeta The TransInfo of which the step stepMeta is part of.
23 | * @param trans The (running) transformation to obtain information shared among the steps.
24 | */
25 | public BeamProduce( StepMeta stepMeta, StepDataInterface stepDataInterface, int copyNr, TransMeta transMeta,
26 | Trans trans ) {
27 | super( stepMeta, stepDataInterface, copyNr, transMeta, trans );
28 | }
29 |
30 | @Override public boolean processRow( StepMetaInterface smi, StepDataInterface sdi ) throws KettleException {
31 |
32 | // Outside of a Beam Runner this step doesn't actually do anything, it's just metadata
33 | // This step gets converted into Beam API calls in a pipeline
34 | //
35 | return false;
36 | }
37 | }
38 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/steps/pubsub/BeamPublish.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.steps.pubsub;
2 |
3 | import org.pentaho.di.core.exception.KettleException;
4 | import org.pentaho.di.trans.Trans;
5 | import org.pentaho.di.trans.TransMeta;
6 | import org.pentaho.di.trans.step.BaseStep;
7 | import org.pentaho.di.trans.step.StepDataInterface;
8 | import org.pentaho.di.trans.step.StepInterface;
9 | import org.pentaho.di.trans.step.StepMeta;
10 | import org.pentaho.di.trans.step.StepMetaInterface;
11 |
12 | public class BeamPublish extends BaseStep implements StepInterface {
13 |
14 | /**
15 | * This is the base step that forms that basis for all steps. You can derive from this class to implement your own
16 | * steps.
17 | *
18 | * @param stepMeta The StepMeta object to run.
19 | * @param stepDataInterface the data object to store temporary data, database connections, caches, result sets,
20 | * hashtables etc.
21 | * @param copyNr The copynumber for this step.
22 | * @param transMeta The TransInfo of which the step stepMeta is part of.
23 | * @param trans The (running) transformation to obtain information shared among the steps.
24 | */
25 | public BeamPublish( StepMeta stepMeta, StepDataInterface stepDataInterface, int copyNr, TransMeta transMeta,
26 | Trans trans ) {
27 | super( stepMeta, stepDataInterface, copyNr, transMeta, trans );
28 | }
29 |
30 | @Override public boolean processRow( StepMetaInterface smi, StepDataInterface sdi ) throws KettleException {
31 |
32 | // Outside of a Beam Runner this step doesn't actually do anything, it's just metadata
33 | // This step gets converted into Beam API calls in a pipeline
34 | //
35 | return false;
36 | }
37 | }
38 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/steps/pubsub/BeamSubscribe.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.steps.pubsub;
2 |
3 | import org.pentaho.di.core.exception.KettleException;
4 | import org.pentaho.di.trans.Trans;
5 | import org.pentaho.di.trans.TransMeta;
6 | import org.pentaho.di.trans.step.BaseStep;
7 | import org.pentaho.di.trans.step.StepDataInterface;
8 | import org.pentaho.di.trans.step.StepInterface;
9 | import org.pentaho.di.trans.step.StepMeta;
10 | import org.pentaho.di.trans.step.StepMetaInterface;
11 |
12 | public class BeamSubscribe extends BaseStep implements StepInterface {
13 |
14 | /**
15 | * This is the base step that forms that basis for all steps. You can derive from this class to implement your own
16 | * steps.
17 | *
18 | * @param stepMeta The StepMeta object to run.
19 | * @param stepDataInterface the data object to store temporary data, database connections, caches, result sets,
20 | * hashtables etc.
21 | * @param copyNr The copynumber for this step.
22 | * @param transMeta The TransInfo of which the step stepMeta is part of.
23 | * @param trans The (running) transformation to obtain information shared among the steps.
24 | */
25 | public BeamSubscribe( StepMeta stepMeta, StepDataInterface stepDataInterface, int copyNr, TransMeta transMeta,
26 | Trans trans ) {
27 | super( stepMeta, stepDataInterface, copyNr, transMeta, trans );
28 | }
29 |
30 | @Override public boolean processRow( StepMetaInterface smi, StepDataInterface sdi ) throws KettleException {
31 |
32 | // Outside of a Beam Runner this step doesn't actually do anything, it's just metadata
33 | // This step gets converted into Beam API calls in a pipeline
34 | //
35 | return false;
36 | }
37 | }
38 |
--------------------------------------------------------------------------------
/src/main/resources/beam_spoon_overlays.xul:
--------------------------------------------------------------------------------
1 |
2 |
6 |
7 |
8 |
11 |
32 |
33 |
34 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/perspective/TransDrawExtensionPoint.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.perspective;
2 |
3 | import org.apache.commons.lang.StringUtils;
4 | import org.kettle.beam.util.BeamConst;
5 | import org.pentaho.di.core.exception.KettleException;
6 | import org.pentaho.di.core.extension.ExtensionPoint;
7 | import org.pentaho.di.core.extension.ExtensionPointInterface;
8 | import org.pentaho.di.core.gui.Point;
9 | import org.pentaho.di.core.gui.PrimitiveGCInterface;
10 | import org.pentaho.di.core.logging.LogChannelInterface;
11 | import org.pentaho.di.trans.TransPainterExtension;
12 |
13 | @ExtensionPoint(
14 | id = "Beam.TransDrawExtensionPoint",
15 | description = "Draw batch or single threaded for Beam",
16 | extensionPointId = "TransPainterStep"
17 | )
18 | public class TransDrawExtensionPoint implements ExtensionPointInterface {
19 | @Override public void callExtensionPoint( LogChannelInterface log, Object object ) throws KettleException {
20 | if ( !( object instanceof TransPainterExtension ) ) {
21 | return;
22 | }
23 | TransPainterExtension ext = (TransPainterExtension) object;
24 | boolean batch = "true".equalsIgnoreCase( ext.stepMeta.getAttribute( BeamConst.STRING_KETTLE_BEAM, BeamConst.STRING_STEP_FLAG_BATCH ) );
25 | boolean single = "true".equalsIgnoreCase( ext.stepMeta.getAttribute( BeamConst.STRING_KETTLE_BEAM, BeamConst.STRING_STEP_FLAG_SINGLE_THREADED ) );
26 | if (!batch && !single) {
27 | return;
28 | }
29 | String str = "";
30 | if ( batch ) {
31 | str += "Batch";
32 | }
33 | if ( single ) {
34 | if ( batch ) {
35 | str += " / ";
36 | }
37 | str += "Single";
38 | }
39 | if ( StringUtils.isNotEmpty( str ) ) {
40 | str="Beam "+str;
41 | Point strSize = ext.gc.textExtent( str );
42 | ext.gc.setFont( PrimitiveGCInterface.EFont.NOTE );
43 | ext.gc.drawText( str, ext.x1 + ext.iconsize, ext.y1 - strSize.y );
44 | }
45 | }
46 | }
47 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/steps/kafka/ConfigOption.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.steps.kafka;
2 |
3 | public class ConfigOption {
4 |
5 | public enum Type {
6 |
7 | String, Short, Int, Long, Double, Boolean,
8 | ;
9 |
10 | public static final Type getTypeFromName(String name) {
11 | for (Type type : values()) {
12 | if (type.name().equalsIgnoreCase( name )) {
13 | return type;
14 | }
15 | }
16 | return String;
17 | }
18 |
19 | public static final String[] getTypeNames() {
20 | String[] names = new String[values().length];
21 | for (int i=0;i PKG = BeamController.class;
33 |
34 | private BindingFactory bf = new SwtBindingFactory();
35 | private Binding revisionBinding;
36 | private Binding changedBinding;
37 | private Binding branchBinding;
38 | private Binding diffBinding;
39 |
40 | public BeamController() {
41 | setName( "beamController" );
42 | }
43 |
44 | public void init() throws IllegalArgumentException, InvocationTargetException, XulException {
45 | XulTextbox diffText = (XulTextbox) document.getElementById( "diff" );
46 | Text text = (Text) diffText.getManagedObject();
47 | text.setFont( JFaceResources.getFont( JFaceResources.TEXT_FONT ) );
48 | }
49 |
50 | private void createBindings() {
51 | // TODO
52 | }
53 |
54 | public void onTabClose() {
55 | // Simply close
56 | }
57 | }
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/core/KettleRow.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.core;
2 |
3 | import java.io.Serializable;
4 |
5 | public class KettleRow implements Serializable {
6 |
7 | private Object[] row;
8 |
9 | public KettleRow() {
10 | }
11 |
12 | public KettleRow( Object[] row ) {
13 | this.row = row;
14 | }
15 |
16 |
17 | @Override public boolean equals( Object obj ) {
18 | if (!(obj instanceof KettleRow )) {
19 | return false;
20 | }
21 | if (this == obj) {
22 | return true;
23 | }
24 | KettleRow otherKettleRow = (KettleRow) obj;
25 |
26 | Object[] thisRow = row;
27 | Object[] otherRow = otherKettleRow.getRow();
28 | if (thisRow==null && otherRow==null) {
29 | return true;
30 | }
31 | if ( (thisRow==null && otherRow!=null) || (thisRow!=null && otherRow==null)) {
32 | return false;
33 | }
34 | if (thisRow.length!=otherRow.length) {
35 | return false;
36 | }
37 | for (int i=0;i(), metaStore );
55 | } catch ( Exception e ) {
56 | throw new KettleException( "There was an error loading step metadata information (loadXML) for step '" + stepname + "'", e );
57 | } finally {
58 | XMLHandlerCache.getInstance().clear();
59 | }
60 | }
61 | }
62 | }
63 |
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 | # kettle-beam
2 | Kettle plugins for Apache Beam
3 |
4 | ## First
5 |
6 | build/install project kettle-beam-core
7 |
8 | https://github.com/mattcasters/kettle-beam-core
9 |
10 |
11 | ## Build
12 |
13 | mvn clean install
14 |
15 | Note you need the Pentaho settings.xml in your ~/.m2 : https://github.com/pentaho/maven-parent-poms/blob/master/maven-support-files/settings.xml
16 |
17 | ## Install
18 |
19 | * Create a new directory called kettle-beam in \plugins/
20 | * Copy target/kettle-beam-.jar to \/plugins/kettle-beam/
21 | * Copy the other jar files in target/lib to \/plugins/kettle-beam/lib/
22 |
23 |
24 | ## Configure
25 |
26 | ### File Definitions
27 |
28 | Describe the file layout for the input and output of your pipeline using :
29 |
30 | Spoon menu Beam / Create a file definition
31 |
32 | Specify this file layout in your "Beam Input" and "Beam Output" steps.
33 | If you do not specify the file definition in the "Beam Output" step, all fields arriving at the step will be written with comma for separator and double quotes as enclosure. The formatting in the fields will be used.
34 |
35 | ### Beam Job Configurations
36 |
37 | A Beam Job configuration is needed to run your transformation on Apache Beam.
38 | Specify which Runner to use (Direct and Dataflow are supported).
39 | You can use the variables to make your transformations completely generic. For example you can set an INPUT_LOCATION location variable
40 | * /some/folder/* for a Direct execution during testing
41 | * gs://mybucket/input/* for an execution on GCP Dataflow
42 |
43 |
44 | ## Supported
45 |
46 | * Input: Beam Input, Google Pub/Sub Subscribe and Google BigQuery Input
47 | * Output: Beam Output, Google Pub/Sub Publish and Google BigQuery Output
48 | * Windowing with the Beam Window step and adding timestamps to bounded data for streaming (Beam Timestamp)
49 | * Sort rows is not yet supported and will never be supported in a generic sense like in Kettle.
50 | * Group By step : experimental, SUM (Integer, Number), COUNT, MIN, MAX, FIRST (throws errors for not-supported stuff)
51 | * Merge Join
52 | * Stream Lookup (side loading data)
53 | * Filter rows (including targeting steps for true/false)
54 | * Switch/Case
55 | * Plugin support through the Beam Job Configuration: specify which plugins to include in the runtime
56 |
57 | ## Runners
58 | * Beam Direct : working
59 | * Google Cloud DataFlow : working
60 | * Apache Spark : mostly untested, configurable (feedback welcome)
61 | * Apache Flink : not started yet, stubbed out code
62 | * Aache Apex : not started yet, stubbed out code
63 | * JStorm : not started yet
64 |
65 | ## More information
66 |
67 | http://diethardsteiner.github.io/pdi/2018/12/01/Kettle-Beam.html
68 |
69 |
70 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/pipeline/handler/BeamBigQueryInputStepHandler.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.pipeline.handler;
2 |
3 | import org.apache.beam.sdk.Pipeline;
4 | import org.apache.beam.sdk.values.PCollection;
5 | import org.kettle.beam.core.KettleRow;
6 | import org.kettle.beam.core.transform.BeamBQInputTransform;
7 | import org.kettle.beam.core.util.JsonRowMeta;
8 | import org.kettle.beam.metastore.BeamJobConfig;
9 | import org.kettle.beam.steps.bq.BeamBQInputMeta;
10 | import org.pentaho.di.core.exception.KettleException;
11 | import org.pentaho.di.core.logging.LogChannelInterface;
12 | import org.pentaho.di.core.row.RowMeta;
13 | import org.pentaho.di.core.row.RowMetaInterface;
14 | import org.pentaho.di.trans.TransMeta;
15 | import org.pentaho.di.trans.step.StepMeta;
16 | import org.pentaho.metastore.api.IMetaStore;
17 |
18 | import java.util.List;
19 | import java.util.Map;
20 |
21 | public class BeamBigQueryInputStepHandler extends BeamBaseStepHandler implements BeamStepHandler {
22 |
23 | public BeamBigQueryInputStepHandler( BeamJobConfig beamJobConfig, IMetaStore metaStore, TransMeta transMeta, List stepPluginClasses, List xpPluginClasses ) {
24 | super( beamJobConfig, true, false, metaStore, transMeta, stepPluginClasses, xpPluginClasses );
25 | }
26 |
27 | @Override public void handleStep( LogChannelInterface log, StepMeta stepMeta, Map> stepCollectionMap,
28 | Pipeline pipeline, RowMetaInterface rowMeta, List previousSteps,
29 | PCollection input ) throws KettleException {
30 |
31 | // Input handling
32 | //
33 | BeamBQInputMeta beamInputMeta = (BeamBQInputMeta) stepMeta.getStepMetaInterface();
34 |
35 | // Output rows (fields selection)
36 | //
37 | RowMetaInterface outputRowMeta = new RowMeta();
38 | beamInputMeta.getFields( outputRowMeta, stepMeta.getName(), null, null, transMeta, null, null );
39 |
40 | BeamBQInputTransform beamInputTransform = new BeamBQInputTransform(
41 | stepMeta.getName(),
42 | stepMeta.getName(),
43 | transMeta.environmentSubstitute( beamInputMeta.getProjectId() ),
44 | transMeta.environmentSubstitute( beamInputMeta.getDatasetId() ),
45 | transMeta.environmentSubstitute( beamInputMeta.getTableId() ),
46 | transMeta.environmentSubstitute( beamInputMeta.getQuery() ),
47 | JsonRowMeta.toJson( outputRowMeta ),
48 | stepPluginClasses,
49 | xpPluginClasses
50 | );
51 | PCollection afterInput = pipeline.apply( beamInputTransform );
52 | stepCollectionMap.put( stepMeta.getName(), afterInput );
53 | log.logBasic( "Handled step (BQ INPUT) : " + stepMeta.getName() );
54 |
55 | }
56 | }
57 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/pipeline/handler/BeamTimestampStepHandler.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.pipeline.handler;
2 |
3 | import org.apache.beam.sdk.Pipeline;
4 | import org.apache.beam.sdk.transforms.ParDo;
5 | import org.apache.beam.sdk.values.PCollection;
6 | import org.apache.commons.lang.StringUtils;
7 | import org.kettle.beam.core.KettleRow;
8 | import org.kettle.beam.core.fn.TimestampFn;
9 | import org.kettle.beam.core.util.JsonRowMeta;
10 | import org.kettle.beam.metastore.BeamJobConfig;
11 | import org.kettle.beam.steps.window.BeamTimestampMeta;
12 | import org.pentaho.di.core.exception.KettleException;
13 | import org.pentaho.di.core.logging.LogChannelInterface;
14 | import org.pentaho.di.core.row.RowMetaInterface;
15 | import org.pentaho.di.trans.TransMeta;
16 | import org.pentaho.di.trans.step.StepMeta;
17 | import org.pentaho.metastore.api.IMetaStore;
18 |
19 | import java.util.List;
20 | import java.util.Map;
21 |
22 | public class BeamTimestampStepHandler extends BeamBaseStepHandler implements BeamStepHandler {
23 |
24 | public BeamTimestampStepHandler( BeamJobConfig beamJobConfig, IMetaStore metaStore, TransMeta transMeta, List stepPluginClasses, List xpPluginClasses ) {
25 | super( beamJobConfig, false, false, metaStore, transMeta, stepPluginClasses, xpPluginClasses );
26 | }
27 |
28 | @Override public void handleStep( LogChannelInterface log, StepMeta stepMeta, Map> stepCollectionMap,
29 | Pipeline pipeline, RowMetaInterface rowMeta, List previousSteps,
30 | PCollection input ) throws KettleException {
31 |
32 | BeamTimestampMeta beamTimestampMeta = (BeamTimestampMeta) stepMeta.getStepMetaInterface();
33 |
34 | if ( !beamTimestampMeta.isReadingTimestamp() && StringUtils.isNotEmpty( beamTimestampMeta.getFieldName() ) ) {
35 | if ( rowMeta.searchValueMeta( beamTimestampMeta.getFieldName() ) == null ) {
36 | throw new KettleException( "Please specify a valid field name '" + stepMeta.getName() + "'" );
37 | }
38 | }
39 |
40 | PCollection stepPCollection = input.apply( ParDo.of(
41 | new TimestampFn(
42 | stepMeta.getName(),
43 | JsonRowMeta.toJson( rowMeta ),
44 | transMeta.environmentSubstitute( beamTimestampMeta.getFieldName() ),
45 | beamTimestampMeta.isReadingTimestamp(),
46 | stepPluginClasses,
47 | xpPluginClasses
48 | ) ) );
49 |
50 |
51 | // Save this in the map
52 | //
53 | stepCollectionMap.put( stepMeta.getName(), stepPCollection );
54 | log.logBasic( "Handled step (TIMESTAMP) : " + stepMeta.getName() + ", gets data from " + previousSteps.size() + " previous step(s)" );
55 | }
56 | }
57 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/pipeline/handler/BeamInputStepHandler.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.pipeline.handler;
2 |
3 | import org.apache.beam.sdk.Pipeline;
4 | import org.apache.beam.sdk.values.PCollection;
5 | import org.kettle.beam.core.KettleRow;
6 | import org.kettle.beam.core.transform.BeamInputTransform;
7 | import org.kettle.beam.core.util.JsonRowMeta;
8 | import org.kettle.beam.metastore.BeamJobConfig;
9 | import org.kettle.beam.metastore.FileDefinition;
10 | import org.kettle.beam.steps.io.BeamInputMeta;
11 | import org.pentaho.di.core.exception.KettleException;
12 | import org.pentaho.di.core.logging.LogChannelInterface;
13 | import org.pentaho.di.core.row.RowMetaInterface;
14 | import org.pentaho.di.trans.TransMeta;
15 | import org.pentaho.di.trans.step.StepMeta;
16 | import org.pentaho.metastore.api.IMetaStore;
17 |
18 | import java.util.List;
19 | import java.util.Map;
20 |
21 | public class BeamInputStepHandler extends BeamBaseStepHandler implements BeamStepHandler {
22 |
23 | public BeamInputStepHandler( BeamJobConfig beamJobConfig, IMetaStore metaStore, TransMeta transMeta, List stepPluginClasses, List xpPluginClasses ) {
24 | super( beamJobConfig, true, false, metaStore, transMeta, stepPluginClasses, xpPluginClasses );
25 | }
26 |
27 | @Override public void handleStep( LogChannelInterface log, StepMeta stepMeta, Map> stepCollectionMap,
28 | Pipeline pipeline, RowMetaInterface rowMeta, List previousSteps,
29 | PCollection input ) throws KettleException {
30 |
31 | // Input handling
32 | //
33 | BeamInputMeta beamInputMeta = (BeamInputMeta) stepMeta.getStepMetaInterface();
34 | FileDefinition inputFileDefinition = beamInputMeta.loadFileDefinition( metaStore );
35 | RowMetaInterface fileRowMeta = inputFileDefinition.getRowMeta();
36 |
37 | // Apply the PBegin to KettleRow transform:
38 | //
39 | if ( inputFileDefinition == null ) {
40 | throw new KettleException( "We couldn't find or load the Beam Input step file definition" );
41 | }
42 | String fileInputLocation = transMeta.environmentSubstitute( beamInputMeta.getInputLocation() );
43 |
44 | BeamInputTransform beamInputTransform = new BeamInputTransform(
45 | stepMeta.getName(),
46 | stepMeta.getName(),
47 | fileInputLocation,
48 | transMeta.environmentSubstitute( inputFileDefinition.getSeparator() ),
49 | JsonRowMeta.toJson( fileRowMeta ),
50 | stepPluginClasses,
51 | xpPluginClasses
52 | );
53 | PCollection afterInput = pipeline.apply( beamInputTransform );
54 | stepCollectionMap.put( stepMeta.getName(), afterInput );
55 | log.logBasic( "Handled step (INPUT) : " + stepMeta.getName() );
56 |
57 | }
58 | }
59 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/pipeline/handler/BeamKafkaOutputStepHandler.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.pipeline.handler;
2 |
3 | import org.apache.beam.sdk.Pipeline;
4 | import org.apache.beam.sdk.values.PCollection;
5 | import org.kettle.beam.core.KettleRow;
6 | import org.kettle.beam.core.transform.BeamKafkaOutputTransform;
7 | import org.kettle.beam.core.util.JsonRowMeta;
8 | import org.kettle.beam.metastore.BeamJobConfig;
9 | import org.kettle.beam.steps.kafka.BeamProduceMeta;
10 | import org.pentaho.di.core.exception.KettleException;
11 | import org.pentaho.di.core.logging.LogChannelInterface;
12 | import org.pentaho.di.core.row.RowMetaInterface;
13 | import org.pentaho.di.trans.TransMeta;
14 | import org.pentaho.di.trans.step.StepMeta;
15 | import org.pentaho.metastore.api.IMetaStore;
16 |
17 | import java.util.List;
18 | import java.util.Map;
19 |
20 | public class BeamKafkaOutputStepHandler extends BeamBaseStepHandler implements BeamStepHandler {
21 |
22 | public BeamKafkaOutputStepHandler( BeamJobConfig beamJobConfig, IMetaStore metaStore, TransMeta transMeta, List stepPluginClasses, List xpPluginClasses ) {
23 | super( beamJobConfig, false, true, metaStore, transMeta, stepPluginClasses, xpPluginClasses );
24 | }
25 |
26 | @Override public void handleStep( LogChannelInterface log, StepMeta beamOutputStepMeta, Map> stepCollectionMap,
27 | Pipeline pipeline, RowMetaInterface rowMeta, List previousSteps,
28 | PCollection input ) throws KettleException {
29 |
30 | BeamProduceMeta beamProduceMeta = (BeamProduceMeta) beamOutputStepMeta.getStepMetaInterface();
31 |
32 | BeamKafkaOutputTransform beamOutputTransform = new BeamKafkaOutputTransform(
33 | beamOutputStepMeta.getName(),
34 | transMeta.environmentSubstitute( beamProduceMeta.getBootstrapServers() ),
35 | transMeta.environmentSubstitute( beamProduceMeta.getTopic() ),
36 | transMeta.environmentSubstitute( beamProduceMeta.getKeyField() ),
37 | transMeta.environmentSubstitute( beamProduceMeta.getMessageField() ),
38 | JsonRowMeta.toJson( rowMeta ),
39 | stepPluginClasses,
40 | xpPluginClasses
41 | );
42 |
43 | // Which step do we apply this transform to?
44 | // Ignore info hops until we figure that out.
45 | //
46 | if ( previousSteps.size() > 1 ) {
47 | throw new KettleException( "Combining data from multiple steps is not supported yet!" );
48 | }
49 | StepMeta previousStep = previousSteps.get( 0 );
50 |
51 | // No need to store this, it's PDone.
52 | //
53 | input.apply( beamOutputTransform );
54 | log.logBasic( "Handled step (KAFKA OUTPUT) : " + beamOutputStepMeta.getName() + ", gets data from " + previousStep.getName() );
55 | }
56 | }
57 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/pipeline/handler/BeamSubscriberStepHandler.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.pipeline.handler;
2 |
3 | import org.apache.beam.sdk.Pipeline;
4 | import org.apache.beam.sdk.values.PCollection;
5 | import org.apache.commons.lang.StringUtils;
6 | import org.kettle.beam.core.KettleRow;
7 | import org.kettle.beam.core.transform.BeamSubscribeTransform;
8 | import org.kettle.beam.core.util.JsonRowMeta;
9 | import org.kettle.beam.metastore.BeamJobConfig;
10 | import org.kettle.beam.steps.pubsub.BeamSubscribeMeta;
11 | import org.pentaho.di.core.exception.KettleException;
12 | import org.pentaho.di.core.logging.LogChannelInterface;
13 | import org.pentaho.di.core.row.RowMetaInterface;
14 | import org.pentaho.di.trans.TransMeta;
15 | import org.pentaho.di.trans.step.StepMeta;
16 | import org.pentaho.metastore.api.IMetaStore;
17 |
18 | import java.util.List;
19 | import java.util.Map;
20 |
21 | public class BeamSubscriberStepHandler extends BeamBaseStepHandler implements BeamStepHandler {
22 |
23 | public BeamSubscriberStepHandler( BeamJobConfig beamJobConfig, IMetaStore metaStore, TransMeta transMeta, List stepPluginClasses, List xpPluginClasses ) {
24 | super( beamJobConfig, true, false, metaStore, transMeta, stepPluginClasses, xpPluginClasses );
25 | }
26 |
27 | @Override public void handleStep( LogChannelInterface log, StepMeta stepMeta, Map> stepCollectionMap,
28 | Pipeline pipeline, RowMetaInterface rowMeta, List previousSteps,
29 | PCollection input ) throws KettleException {
30 |
31 | // A Beam subscriber step
32 | //
33 | BeamSubscribeMeta inputMeta = (BeamSubscribeMeta) stepMeta.getStepMetaInterface();
34 |
35 | RowMetaInterface outputRowMeta = transMeta.getStepFields( stepMeta );
36 | String rowMetaJson = JsonRowMeta.toJson( outputRowMeta );
37 |
38 | // Verify some things:
39 | //
40 | if ( StringUtils.isEmpty( inputMeta.getTopic() ) ) {
41 | throw new KettleException( "Please specify a topic to read from in Beam Pub/Sub Subscribe step '" + stepMeta.getName() + "'" );
42 | }
43 |
44 | BeamSubscribeTransform subscribeTransform = new BeamSubscribeTransform(
45 | stepMeta.getName(),
46 | stepMeta.getName(),
47 | transMeta.environmentSubstitute( inputMeta.getSubscription() ),
48 | transMeta.environmentSubstitute( inputMeta.getTopic() ),
49 | inputMeta.getMessageType(),
50 | rowMetaJson,
51 | stepPluginClasses,
52 | xpPluginClasses
53 | );
54 |
55 | PCollection afterInput = pipeline.apply( subscribeTransform );
56 | stepCollectionMap.put( stepMeta.getName(), afterInput );
57 |
58 | log.logBasic( "Handled step (SUBSCRIBE) : " + stepMeta.getName() );
59 | }
60 | }
61 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/pipeline/handler/BeamGroupByStepHandler.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.pipeline.handler;
2 |
3 | import org.apache.beam.sdk.Pipeline;
4 | import org.apache.beam.sdk.transforms.PTransform;
5 | import org.apache.beam.sdk.values.PCollection;
6 | import org.kettle.beam.core.KettleRow;
7 | import org.kettle.beam.core.transform.GroupByTransform;
8 | import org.kettle.beam.core.util.JsonRowMeta;
9 | import org.kettle.beam.metastore.BeamJobConfig;
10 | import org.pentaho.di.core.exception.KettleException;
11 | import org.pentaho.di.core.logging.LogChannelInterface;
12 | import org.pentaho.di.core.row.RowMetaInterface;
13 | import org.pentaho.di.trans.TransMeta;
14 | import org.pentaho.di.trans.step.StepMeta;
15 | import org.pentaho.di.trans.steps.memgroupby.MemoryGroupByMeta;
16 | import org.pentaho.metastore.api.IMetaStore;
17 |
18 | import java.util.List;
19 | import java.util.Map;
20 |
21 | public class BeamGroupByStepHandler extends BeamBaseStepHandler implements BeamStepHandler {
22 |
23 | public BeamGroupByStepHandler( BeamJobConfig beamJobConfig, IMetaStore metaStore, TransMeta transMeta, List stepPluginClasses, List xpPluginClasses ) {
24 | super( beamJobConfig, false, false, metaStore, transMeta, stepPluginClasses, xpPluginClasses );
25 | }
26 |
27 | @Override public void handleStep( LogChannelInterface log, StepMeta stepMeta, Map> stepCollectionMap,
28 | Pipeline pipeline, RowMetaInterface rowMeta, List previousSteps,
29 | PCollection input ) throws KettleException {
30 |
31 | MemoryGroupByMeta groupByMeta = (MemoryGroupByMeta) stepMeta.getStepMetaInterface();
32 |
33 | String[] aggregates = new String[ groupByMeta.getAggregateType().length ];
34 | for ( int i = 0; i < aggregates.length; i++ ) {
35 | aggregates[ i ] = MemoryGroupByMeta.getTypeDesc( groupByMeta.getAggregateType()[ i ] );
36 | }
37 |
38 | PTransform, PCollection> stepTransform = new GroupByTransform(
39 | stepMeta.getName(),
40 | JsonRowMeta.toJson( rowMeta ), // The io row
41 | stepPluginClasses,
42 | xpPluginClasses,
43 | groupByMeta.getGroupField(),
44 | groupByMeta.getSubjectField(),
45 | aggregates,
46 | groupByMeta.getAggregateField()
47 | );
48 |
49 | // Apply the step transform to the previous io step PCollection(s)
50 | //
51 | PCollection stepPCollection = input.apply( stepMeta.getName(), stepTransform );
52 |
53 | // Save this in the map
54 | //
55 | stepCollectionMap.put( stepMeta.getName(), stepPCollection );
56 | log.logBasic( "Handled Group By (STEP) : " + stepMeta.getName() + ", gets data from " + previousSteps.size() + " previous step(s)" );
57 | }
58 | }
59 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/perspective/BeamSpoonPlugin.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.perspective;
2 |
3 | /*
4 | * Copyright 2017 Hitachi America, Ltd., R&D.
5 | *
6 | * Licensed under the Apache License, Version 2.0 (the "License");
7 | * you may not use this file except in compliance with the License.
8 | * 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 | import java.util.ResourceBundle;
20 |
21 | import org.pentaho.di.ui.spoon.SpoonLifecycleListener;
22 | import org.pentaho.di.ui.spoon.SpoonPerspective;
23 | import org.pentaho.di.ui.spoon.SpoonPlugin;
24 | import org.pentaho.di.ui.spoon.SpoonPluginCategories;
25 | import org.pentaho.di.ui.spoon.SpoonPluginInterface;
26 | import org.pentaho.di.ui.spoon.XulSpoonResourceBundle;
27 | import org.pentaho.ui.xul.XulDomContainer;
28 | import org.pentaho.ui.xul.XulException;
29 |
30 | @SpoonPlugin( id = "BeamSpoonPlugin", image = "" )
31 | @SpoonPluginCategories( { "spoon", "trans-graph" } )
32 | public class BeamSpoonPlugin implements SpoonPluginInterface, SpoonLifecycleListener {
33 |
34 | private static final Class> PKG = BeamSpoonPlugin.class;
35 | private ResourceBundle resourceBundle = new XulSpoonResourceBundle( PKG );
36 |
37 | // private BeamPerspective perspective;
38 |
39 | public BeamSpoonPlugin() throws XulException {
40 | // this.perspective = new BeamPerspective();
41 | }
42 |
43 | @Override
44 | public void onEvent( SpoonLifeCycleEvent evt ) {
45 | // TODO Auto-generated method stub
46 | }
47 |
48 | @Override
49 | public void applyToContainer( String category, XulDomContainer container ) throws XulException {
50 | container.registerClassLoader( getClass().getClassLoader() );
51 | if ( category.equals( "spoon" ) ) {
52 | container.loadOverlay( "beam_spoon_overlays.xul", resourceBundle );
53 | container.addEventHandler( BeamHelper.getInstance() );
54 | }
55 | if ( category.equals( "trans-graph" ) ) {
56 | container.loadOverlay( "beam_transgraph_overlays.xul", resourceBundle );
57 | container.addEventHandler( BeamHelper.getInstance() );
58 | }
59 | }
60 |
61 | @Override
62 | public SpoonLifecycleListener getLifecycleListener() {
63 | // TODO Auto-generated method stub
64 | return this;
65 | }
66 |
67 | @Override public SpoonPerspective getPerspective() {
68 | return null;
69 | }
70 |
71 | /*
72 | @Override
73 | public SpoonPerspective getPerspective() {
74 | return perspective;
75 | }
76 | */
77 |
78 | }
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/core/fn/StringToKettleRowFn.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.core.fn;
2 |
3 | import org.apache.beam.sdk.metrics.Counter;
4 | import org.apache.beam.sdk.metrics.Metrics;
5 | import org.apache.beam.sdk.transforms.DoFn;
6 | import org.kettle.beam.core.BeamKettle;
7 | import org.kettle.beam.core.KettleRow;
8 | import org.kettle.beam.core.util.JsonRowMeta;
9 | import org.pentaho.di.core.row.RowDataUtil;
10 | import org.pentaho.di.core.row.RowMetaInterface;
11 | import org.slf4j.Logger;
12 | import org.slf4j.LoggerFactory;
13 |
14 | import java.util.List;
15 |
16 | public class StringToKettleRowFn extends DoFn {
17 |
18 | private String rowMetaJson;
19 | private String stepname;
20 | private List stepPluginClasses;
21 | private List xpPluginClasses;
22 |
23 | private static final Logger LOG = LoggerFactory.getLogger( StringToKettleRowFn.class );
24 | private final Counter numErrors = Metrics.counter( "main", "BeamSubscribeTransformErrors" );
25 |
26 | private RowMetaInterface rowMeta;
27 | private transient Counter initCounter;
28 | private transient Counter inputCounter;
29 | private transient Counter writtenCounter;
30 |
31 | public StringToKettleRowFn( String stepname, String rowMetaJson, List stepPluginClasses, List xpPluginClasses ) {
32 | this.stepname = stepname;
33 | this.rowMetaJson = rowMetaJson;
34 | this.stepPluginClasses = stepPluginClasses;
35 | this.xpPluginClasses = xpPluginClasses;
36 | }
37 |
38 | @Setup
39 | public void setUp() {
40 | try {
41 | inputCounter = Metrics.counter( "input", stepname );
42 | writtenCounter = Metrics.counter( "written", stepname );
43 |
44 | // Initialize Kettle Beam
45 | //
46 | BeamKettle.init( stepPluginClasses, xpPluginClasses );
47 | rowMeta = JsonRowMeta.fromJson( rowMetaJson );
48 |
49 | Metrics.counter( "init", stepname ).inc();
50 | } catch(Exception e) {
51 | numErrors.inc();
52 | LOG.error( "Error in setup of String to Kettle Row conversion function", e );
53 | throw new RuntimeException( "Error in setup of String to Kettle Row conversion function", e );
54 | }
55 | }
56 |
57 | @ProcessElement
58 | public void processElement( ProcessContext processContext ) {
59 | try {
60 |
61 | String string = processContext.element();
62 | inputCounter.inc();
63 |
64 | Object[] outputRow = RowDataUtil.allocateRowData( rowMeta.size() );
65 | outputRow[ 0 ] = string;
66 |
67 | processContext.output( new KettleRow( outputRow ) );
68 | writtenCounter.inc();
69 |
70 | } catch ( Exception e ) {
71 | numErrors.inc();
72 | LOG.error( "Error in String to Kettle Row conversion function", e );
73 | throw new RuntimeException( "Error in String to Kettle Row conversion function", e );
74 | }
75 | }
76 | }
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/pipeline/handler/BeamPublisherStepHandler.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.pipeline.handler;
2 |
3 | import org.apache.beam.sdk.Pipeline;
4 | import org.apache.beam.sdk.values.PCollection;
5 | import org.apache.commons.lang.StringUtils;
6 | import org.kettle.beam.core.KettleRow;
7 | import org.kettle.beam.core.transform.BeamPublishTransform;
8 | import org.kettle.beam.core.util.JsonRowMeta;
9 | import org.kettle.beam.metastore.BeamJobConfig;
10 | import org.kettle.beam.steps.pubsub.BeamPublishMeta;
11 | import org.pentaho.di.core.exception.KettleException;
12 | import org.pentaho.di.core.logging.LogChannelInterface;
13 | import org.pentaho.di.core.row.RowMetaInterface;
14 | import org.pentaho.di.trans.TransMeta;
15 | import org.pentaho.di.trans.step.StepMeta;
16 | import org.pentaho.metastore.api.IMetaStore;
17 |
18 | import java.util.List;
19 | import java.util.Map;
20 |
21 | public class BeamPublisherStepHandler extends BeamBaseStepHandler implements BeamStepHandler {
22 |
23 | public BeamPublisherStepHandler( BeamJobConfig beamJobConfig, IMetaStore metaStore, TransMeta transMeta, List stepPluginClasses, List xpPluginClasses ) {
24 | super( beamJobConfig, false, true, metaStore, transMeta, stepPluginClasses, xpPluginClasses );
25 | }
26 |
27 | @Override public void handleStep( LogChannelInterface log, StepMeta stepMeta, Map> stepCollectionMap,
28 | Pipeline pipeline, RowMetaInterface rowMeta, List previousSteps,
29 | PCollection input ) throws KettleException {
30 |
31 | BeamPublishMeta publishMeta = (BeamPublishMeta) stepMeta.getStepMetaInterface();
32 |
33 | // some validation
34 | //
35 | if ( StringUtils.isEmpty( publishMeta.getTopic() ) ) {
36 | throw new KettleException( "Please specify a topic to publish to in Beam Pub/Sub Publish step '" + stepMeta.getName() + "'" );
37 | }
38 |
39 | BeamPublishTransform beamOutputTransform = new BeamPublishTransform(
40 | stepMeta.getName(),
41 | transMeta.environmentSubstitute( publishMeta.getTopic() ),
42 | publishMeta.getMessageType(),
43 | publishMeta.getMessageField(),
44 | JsonRowMeta.toJson( rowMeta ),
45 | stepPluginClasses,
46 | xpPluginClasses
47 | );
48 |
49 | // Which step do we apply this transform to?
50 | // Ignore info hops until we figure that out.
51 | //
52 | if ( previousSteps.size() > 1 ) {
53 | throw new KettleException( "Combining data from multiple steps is not supported yet!" );
54 | }
55 | StepMeta previousStep = previousSteps.get( 0 );
56 |
57 | // No need to store this, it's PDone.
58 | //
59 | input.apply( beamOutputTransform );
60 | log.logBasic( "Handled step (PUBLISH) : " + stepMeta.getName() + ", gets data from " + previousStep.getName() );
61 | }
62 | }
63 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/core/fn/PublishStringsFn.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.core.fn;
2 |
3 | import org.apache.beam.sdk.metrics.Counter;
4 | import org.apache.beam.sdk.metrics.Metrics;
5 | import org.apache.beam.sdk.transforms.DoFn;
6 | import org.kettle.beam.core.BeamKettle;
7 | import org.kettle.beam.core.KettleRow;
8 | import org.kettle.beam.core.util.JsonRowMeta;
9 | import org.pentaho.di.core.row.RowMetaInterface;
10 | import org.slf4j.Logger;
11 | import org.slf4j.LoggerFactory;
12 |
13 | import java.util.List;
14 |
15 | public class PublishStringsFn extends DoFn {
16 |
17 | private String rowMetaJson;
18 | private int fieldIndex;
19 | private String stepname;
20 | private List stepPluginClasses;
21 | private List xpPluginClasses;
22 |
23 | private static final Logger LOG = LoggerFactory.getLogger( PublishStringsFn.class );
24 | private final Counter numErrors = Metrics.counter( "main", "BeamPublishTransformErrors" );
25 |
26 | private RowMetaInterface rowMeta;
27 | private transient Counter initCounter;
28 | private transient Counter readCounter;
29 | private transient Counter outputCounter;
30 |
31 | public PublishStringsFn( String stepname, int fieldIndex, String rowMetaJson, List stepPluginClasses, List xpPluginClasses ) {
32 | this.stepname = stepname;
33 | this.fieldIndex = fieldIndex;
34 | this.rowMetaJson = rowMetaJson;
35 | this.stepPluginClasses = stepPluginClasses;
36 | this.xpPluginClasses = xpPluginClasses;
37 | }
38 |
39 | @Setup
40 | public void setUp() {
41 | try {
42 | readCounter = Metrics.counter( "read", stepname );
43 | outputCounter = Metrics.counter( "output", stepname );
44 |
45 | // Initialize Kettle Beam
46 | //
47 | BeamKettle.init( stepPluginClasses, xpPluginClasses );
48 | rowMeta = JsonRowMeta.fromJson( rowMetaJson );
49 |
50 | Metrics.counter( "init", stepname ).inc();
51 | } catch ( Exception e ) {
52 | numErrors.inc();
53 | LOG.error( "Error in setup of pub/sub publish messages function", e );
54 | throw new RuntimeException( "Error in setup of pub/sub publish messages function", e );
55 | }
56 | }
57 |
58 | @ProcessElement
59 | public void processElement( ProcessContext processContext ) {
60 |
61 | try {
62 |
63 | KettleRow kettleRow = processContext.element();
64 | readCounter.inc();
65 |
66 | try {
67 | String string = rowMeta.getString( kettleRow.getRow(), fieldIndex );
68 | processContext.output( string );
69 | outputCounter.inc();
70 | } catch ( Exception e ) {
71 | throw new RuntimeException( "Unable to pass string", e );
72 | }
73 |
74 | } catch ( Exception e ) {
75 | numErrors.inc();
76 | LOG.error( "Error in pub/sub publish messages function", e );
77 | throw new RuntimeException( "Error in pub/sub publish messages function", e );
78 | }
79 | }
80 | }
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/core/fn/PubsubMessageToKettleRowFn.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.core.fn;
2 |
3 | import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage;
4 | import org.apache.beam.sdk.metrics.Counter;
5 | import org.apache.beam.sdk.metrics.Metrics;
6 | import org.apache.beam.sdk.transforms.DoFn;
7 | import org.kettle.beam.core.BeamKettle;
8 | import org.kettle.beam.core.KettleRow;
9 | import org.kettle.beam.core.util.JsonRowMeta;
10 | import org.pentaho.di.core.row.RowDataUtil;
11 | import org.pentaho.di.core.row.RowMetaInterface;
12 | import org.slf4j.Logger;
13 | import org.slf4j.LoggerFactory;
14 |
15 | import java.util.List;
16 |
17 | public class PubsubMessageToKettleRowFn extends DoFn {
18 |
19 | private String rowMetaJson;
20 | private String stepname;
21 | private List stepPluginClasses;
22 | private List xpPluginClasses;
23 |
24 | private static final Logger LOG = LoggerFactory.getLogger( PubsubMessageToKettleRowFn.class );
25 | private final Counter numErrors = Metrics.counter( "main", "BeamSubscribeTransformErrors" );
26 |
27 | private RowMetaInterface rowMeta;
28 | private transient Counter initCounter;
29 | private transient Counter inputCounter;
30 | private transient Counter writtenCounter;
31 |
32 | public PubsubMessageToKettleRowFn( String stepname, String rowMetaJson, List stepPluginClasses, List xpPluginClasses ) {
33 | this.stepname = stepname;
34 | this.rowMetaJson = rowMetaJson;
35 | this.stepPluginClasses = stepPluginClasses;
36 | this.xpPluginClasses = xpPluginClasses;
37 | }
38 |
39 | @Setup
40 | public void setUp() {
41 | try {
42 | inputCounter = Metrics.counter( "input", stepname );
43 | writtenCounter = Metrics.counter( "written", stepname );
44 |
45 | // Initialize Kettle Beam
46 | //
47 | BeamKettle.init( stepPluginClasses, xpPluginClasses );
48 | rowMeta = JsonRowMeta.fromJson( rowMetaJson );
49 |
50 | Metrics.counter( "init", stepname ).inc();
51 | } catch ( Exception e ) {
52 | numErrors.inc();
53 | LOG.error( "Error in setup of pub/sub publish messages function", e );
54 | throw new RuntimeException( "Error in setup of pub/sub publish messages function", e );
55 | }
56 | }
57 |
58 | @ProcessElement
59 | public void processElement( ProcessContext processContext ) {
60 | try {
61 |
62 | PubsubMessage message = processContext.element();
63 | inputCounter.inc();
64 |
65 | Object[] outputRow = RowDataUtil.allocateRowData(rowMeta.size());
66 | outputRow[0] = message; // Serializable
67 |
68 | processContext.output( new KettleRow( outputRow ) );
69 | writtenCounter.inc();
70 |
71 | } catch ( Exception e ) {
72 | numErrors.inc();
73 | LOG.error( "Error in pub/sub publish messages function", e );
74 | throw new RuntimeException( "Error in pub/sub publish messages function", e );
75 | }
76 | }
77 | }
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/carte/BeamMetricsEntry.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.carte;
2 |
3 | import java.util.Date;
4 |
5 | import org.pentaho.di.www.SlaveServerTransStatus;
6 |
7 | public class BeamMetricsEntry {
8 | /** the carte Object ID of the transformation, some unique number */
9 | private String carteObjectId;
10 |
11 | /** the name of the transformation */
12 | private String trans;
13 |
14 | /** The internal Job ID (Spark, Flink, ...)*/
15 | private String internalJobId;
16 |
17 | /** The update date */
18 | private Date updateDate;
19 |
20 | /**
21 | * The transformation status.
22 | */
23 | private SlaveServerTransStatus transStatus;
24 |
25 | public BeamMetricsEntry() {
26 | }
27 |
28 | public BeamMetricsEntry( String carteObjectId, String trans, String internalJobId, Date updateDate, SlaveServerTransStatus transStatus ) {
29 | this.carteObjectId = carteObjectId;
30 | this.trans = trans;
31 | this.internalJobId = internalJobId;
32 | this.updateDate = updateDate;
33 | this.transStatus = transStatus;
34 | }
35 |
36 | /**
37 | * Gets carteObjectId
38 | *
39 | * @return value of carteObjectId
40 | */
41 | public String getCarteObjectId() {
42 | return carteObjectId;
43 | }
44 |
45 | /**
46 | * @param carteObjectId The carteObjectId to set
47 | */
48 | public void setCarteObjectId( String carteObjectId ) {
49 | this.carteObjectId = carteObjectId;
50 | }
51 |
52 | /**
53 | * Gets trans
54 | *
55 | * @return value of trans
56 | */
57 | public String getTrans() {
58 | return trans;
59 | }
60 |
61 | /**
62 | * @param trans The trans to set
63 | */
64 | public void setTrans( String trans ) {
65 | this.trans = trans;
66 | }
67 |
68 | /**
69 | * Gets internalJobId
70 | *
71 | * @return value of internalJobId
72 | */
73 | public String getInternalJobId() {
74 | return internalJobId;
75 | }
76 |
77 | /**
78 | * @param internalJobId The internalJobId to set
79 | */
80 | public void setInternalJobId( String internalJobId ) {
81 | this.internalJobId = internalJobId;
82 | }
83 |
84 | /**
85 | * Gets transStatus
86 | *
87 | * @return value of transStatus
88 | */
89 | public SlaveServerTransStatus getTransStatus() {
90 | return transStatus;
91 | }
92 |
93 | /**
94 | * @param transStatus The transStatus to set
95 | */
96 | public void setTransStatus( SlaveServerTransStatus transStatus ) {
97 | this.transStatus = transStatus;
98 | }
99 |
100 | /**
101 | * Gets updateDate
102 | *
103 | * @return value of updateDate
104 | */
105 | public Date getUpdateDate() {
106 | return updateDate;
107 | }
108 |
109 | /**
110 | * @param updateDate The updateDate to set
111 | */
112 | public void setUpdateDate( Date updateDate ) {
113 | this.updateDate = updateDate;
114 | }
115 | }
116 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/core/fn/KVLongStringToKettleRowFn.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.core.fn;
2 |
3 | import org.apache.beam.sdk.metrics.Counter;
4 | import org.apache.beam.sdk.metrics.Metrics;
5 | import org.apache.beam.sdk.transforms.DoFn;
6 | import org.apache.beam.sdk.values.KV;
7 | import org.kettle.beam.core.BeamKettle;
8 | import org.kettle.beam.core.KettleRow;
9 | import org.kettle.beam.core.util.JsonRowMeta;
10 | import org.pentaho.di.core.row.RowDataUtil;
11 | import org.pentaho.di.core.row.RowMetaInterface;
12 | import org.slf4j.Logger;
13 | import org.slf4j.LoggerFactory;
14 |
15 | import java.util.List;
16 |
17 | public class KVLongStringToKettleRowFn extends DoFn, KettleRow> {
18 |
19 | private String rowMetaJson;
20 | private String stepname;
21 | private List stepPluginClasses;
22 | private List xpPluginClasses;
23 |
24 | private static final Logger LOG = LoggerFactory.getLogger( KVLongStringToKettleRowFn.class );
25 | private final Counter numErrors = Metrics.counter( "main", "BeamSubscribeTransformErrors" );
26 |
27 | private RowMetaInterface rowMeta;
28 | private transient Counter initCounter;
29 | private transient Counter inputCounter;
30 | private transient Counter writtenCounter;
31 |
32 | public KVLongStringToKettleRowFn( String stepname, String rowMetaJson, List stepPluginClasses, List xpPluginClasses ) {
33 | this.stepname = stepname;
34 | this.rowMetaJson = rowMetaJson;
35 | this.stepPluginClasses = stepPluginClasses;
36 | this.xpPluginClasses = xpPluginClasses;
37 | }
38 |
39 | @Setup
40 | public void setUp() {
41 | try {
42 | inputCounter = Metrics.counter( "input", stepname );
43 | writtenCounter = Metrics.counter( "written", stepname );
44 |
45 | // Initialize Kettle Beam
46 | //
47 | BeamKettle.init( stepPluginClasses, xpPluginClasses );
48 | rowMeta = JsonRowMeta.fromJson( rowMetaJson );
49 |
50 | Metrics.counter( "init", stepname ).inc();
51 | } catch ( Exception e ) {
52 | numErrors.inc();
53 | LOG.error( "Error in setup of KV to Kettle Row conversion function", e );
54 | throw new RuntimeException( "Error in setup of KV to Kettle Row conversion function", e );
55 | }
56 | }
57 |
58 | @ProcessElement
59 | public void processElement( ProcessContext processContext ) {
60 | try {
61 |
62 | KV kv = processContext.element();
63 | inputCounter.inc();
64 |
65 | Object[] outputRow = RowDataUtil.allocateRowData( rowMeta.size() );
66 | outputRow[ 0 ] = kv.getKey();
67 | outputRow[ 1 ] = kv.getValue();
68 |
69 | processContext.output( new KettleRow( outputRow ) );
70 | writtenCounter.inc();
71 |
72 | } catch ( Exception e ) {
73 | numErrors.inc();
74 | LOG.error( "Error in KV to Kettle Row conversion function", e );
75 | throw new RuntimeException( "Error in KV to Kettle Row conversion function", e );
76 | }
77 | }
78 | }
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/core/fn/KVStringStringToKettleRowFn.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.core.fn;
2 |
3 | import org.apache.beam.sdk.metrics.Counter;
4 | import org.apache.beam.sdk.metrics.Metrics;
5 | import org.apache.beam.sdk.transforms.DoFn;
6 | import org.apache.beam.sdk.values.KV;
7 | import org.kettle.beam.core.BeamKettle;
8 | import org.kettle.beam.core.KettleRow;
9 | import org.kettle.beam.core.util.JsonRowMeta;
10 | import org.pentaho.di.core.row.RowDataUtil;
11 | import org.pentaho.di.core.row.RowMetaInterface;
12 | import org.slf4j.Logger;
13 | import org.slf4j.LoggerFactory;
14 |
15 | import java.util.List;
16 |
17 | public class KVStringStringToKettleRowFn extends DoFn, KettleRow> {
18 |
19 | private String rowMetaJson;
20 | private String stepname;
21 | private List stepPluginClasses;
22 | private List xpPluginClasses;
23 |
24 | private static final Logger LOG = LoggerFactory.getLogger( KVStringStringToKettleRowFn.class );
25 | private final Counter numErrors = Metrics.counter( "main", "BeamSubscribeTransformErrors" );
26 |
27 | private RowMetaInterface rowMeta;
28 | private transient Counter initCounter;
29 | private transient Counter inputCounter;
30 | private transient Counter writtenCounter;
31 |
32 | public KVStringStringToKettleRowFn( String stepname, String rowMetaJson, List stepPluginClasses, List xpPluginClasses ) {
33 | this.stepname = stepname;
34 | this.rowMetaJson = rowMetaJson;
35 | this.stepPluginClasses = stepPluginClasses;
36 | this.xpPluginClasses = xpPluginClasses;
37 | }
38 |
39 | @Setup
40 | public void setUp() {
41 | try {
42 | inputCounter = Metrics.counter( "input", stepname );
43 | writtenCounter = Metrics.counter( "written", stepname );
44 |
45 | // Initialize Kettle Beam
46 | //
47 | BeamKettle.init( stepPluginClasses, xpPluginClasses );
48 | rowMeta = JsonRowMeta.fromJson( rowMetaJson );
49 |
50 | Metrics.counter( "init", stepname ).inc();
51 | } catch ( Exception e ) {
52 | numErrors.inc();
53 | LOG.error( "Error in setup of KV to Kettle Row conversion function", e );
54 | throw new RuntimeException( "Error in setup of KV to Kettle Row conversion function", e );
55 | }
56 | }
57 |
58 | @ProcessElement
59 | public void processElement( ProcessContext processContext ) {
60 | try {
61 |
62 | KV kv = processContext.element();
63 | inputCounter.inc();
64 |
65 | Object[] outputRow = RowDataUtil.allocateRowData( rowMeta.size() );
66 | outputRow[ 0 ] = kv.getKey(); // String
67 | outputRow[ 1 ] = kv.getValue(); // String
68 |
69 | processContext.output( new KettleRow( outputRow ) );
70 | writtenCounter.inc();
71 |
72 | } catch ( Exception e ) {
73 | numErrors.inc();
74 | LOG.error( "Error in KV to Kettle Row conversion function", e );
75 | throw new RuntimeException( "Error in KV to Kettle Row conversion function", e );
76 | }
77 | }
78 | }
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/core/fn/KettleRowToKVStringStringFn.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.core.fn;
2 |
3 | import org.apache.beam.sdk.metrics.Counter;
4 | import org.apache.beam.sdk.metrics.Metrics;
5 | import org.apache.beam.sdk.transforms.DoFn;
6 | import org.apache.beam.sdk.values.KV;
7 | import org.kettle.beam.core.BeamKettle;
8 | import org.kettle.beam.core.KettleRow;
9 | import org.kettle.beam.core.util.JsonRowMeta;
10 | import org.pentaho.di.core.row.RowMetaInterface;
11 | import org.slf4j.Logger;
12 | import org.slf4j.LoggerFactory;
13 |
14 | import java.util.List;
15 |
16 | public class KettleRowToKVStringStringFn extends DoFn> {
17 |
18 | private String rowMetaJson;
19 | private String stepname;
20 | private int keyIndex;
21 | private int valueIndex;
22 | private List stepPluginClasses;
23 | private List xpPluginClasses;
24 |
25 | private static final Logger LOG = LoggerFactory.getLogger( KettleRowToKVStringStringFn.class );
26 | private final Counter numErrors = Metrics.counter( "main", "BeamSubscribeTransformErrors" );
27 |
28 | private RowMetaInterface rowMeta;
29 | private transient Counter initCounter;
30 | private transient Counter inputCounter;
31 | private transient Counter writtenCounter;
32 |
33 | public KettleRowToKVStringStringFn( String stepname, int keyIndex, int valueIndex, String rowMetaJson, List stepPluginClasses, List xpPluginClasses ) {
34 | this.stepname = stepname;
35 | this.keyIndex = keyIndex;
36 | this.valueIndex = valueIndex;
37 | this.rowMetaJson = rowMetaJson;
38 | this.stepPluginClasses = stepPluginClasses;
39 | this.xpPluginClasses = xpPluginClasses;
40 | }
41 |
42 | @Setup
43 | public void setUp() {
44 | try {
45 | inputCounter = Metrics.counter( "input", stepname );
46 | writtenCounter = Metrics.counter( "written", stepname );
47 |
48 | // Initialize Kettle Beam
49 | //
50 | BeamKettle.init( stepPluginClasses, xpPluginClasses );
51 | rowMeta = JsonRowMeta.fromJson( rowMetaJson );
52 |
53 | Metrics.counter( "init", stepname ).inc();
54 | } catch ( Exception e ) {
55 | numErrors.inc();
56 | LOG.error( "Error in setup of KettleRow to KV function", e );
57 | throw new RuntimeException( "Error in setup of KettleRow to KV function", e );
58 | }
59 | }
60 |
61 | @ProcessElement
62 | public void processElement( ProcessContext processContext ) {
63 | try {
64 | KettleRow kettleRow = processContext.element();
65 | inputCounter.inc();
66 |
67 | String key = rowMeta.getString(kettleRow.getRow(), keyIndex);
68 | String value = rowMeta.getString(kettleRow.getRow(), valueIndex);
69 |
70 | processContext.output( KV.of( key, value ) );
71 | writtenCounter.inc();
72 |
73 | } catch ( Exception e ) {
74 | numErrors.inc();
75 | LOG.error( "Error in KettleRow to KV function", e );
76 | throw new RuntimeException( "Error in KettleRow to KV function", e );
77 | }
78 | }
79 | }
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/core/fn/PublishMessagesFn.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.core.fn;
2 |
3 | import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage;
4 | import org.apache.beam.sdk.metrics.Counter;
5 | import org.apache.beam.sdk.metrics.Metrics;
6 | import org.apache.beam.sdk.transforms.DoFn;
7 | import org.kettle.beam.core.BeamKettle;
8 | import org.kettle.beam.core.KettleRow;
9 | import org.kettle.beam.core.util.JsonRowMeta;
10 | import org.pentaho.di.core.row.RowMetaInterface;
11 | import org.slf4j.Logger;
12 | import org.slf4j.LoggerFactory;
13 |
14 | import java.util.HashMap;
15 | import java.util.List;
16 |
17 | public class PublishMessagesFn extends DoFn {
18 |
19 | private String rowMetaJson;
20 | private int fieldIndex;
21 | private String stepname;
22 | private List stepPluginClasses;
23 | private List xpPluginClasses;
24 |
25 | private static final Logger LOG = LoggerFactory.getLogger( PublishMessagesFn.class );
26 | private final Counter numErrors = Metrics.counter( "main", "BeamPublishTransformErrors" );
27 |
28 | private RowMetaInterface rowMeta;
29 | private transient Counter initCounter;
30 | private transient Counter readCounter;
31 | private transient Counter outputCounter;
32 |
33 | public PublishMessagesFn( String stepname, int fieldIndex, String rowMetaJson, List stepPluginClasses, List xpPluginClasses ) {
34 | this.stepname = stepname;
35 | this.fieldIndex = fieldIndex;
36 | this.rowMetaJson = rowMetaJson;
37 | this.stepPluginClasses = stepPluginClasses;
38 | this.xpPluginClasses = xpPluginClasses;
39 | }
40 |
41 | @Setup
42 | public void setUp() {
43 | try {
44 | readCounter = Metrics.counter( "read", stepname );
45 | outputCounter = Metrics.counter( "output", stepname );
46 |
47 | // Initialize Kettle Beam
48 | //
49 | BeamKettle.init( stepPluginClasses, xpPluginClasses );
50 | rowMeta = JsonRowMeta.fromJson( rowMetaJson );
51 |
52 | Metrics.counter( "init", stepname ).inc();
53 | } catch ( Exception e ) {
54 | numErrors.inc();
55 | LOG.error( "Error in setup of pub/sub publish messages function", e );
56 | throw new RuntimeException( "Error in setup of pub/sub publish messages function", e );
57 | }
58 | }
59 |
60 | @ProcessElement
61 | public void processElement( ProcessContext processContext ) {
62 |
63 | try {
64 | KettleRow kettleRow = processContext.element();
65 | readCounter.inc();
66 | try {
67 | byte[] bytes = rowMeta.getBinary( kettleRow.getRow(), fieldIndex );
68 | PubsubMessage message = new PubsubMessage( bytes, new HashMap<>() );
69 | processContext.output( message );
70 | outputCounter.inc();
71 | } catch ( Exception e ) {
72 | throw new RuntimeException( "Unable to pass message", e );
73 | }
74 |
75 | } catch ( Exception e ) {
76 | numErrors.inc();
77 | LOG.error( "Error in pub/sub publish messages function", e );
78 | throw new RuntimeException( "Error in pub/sub publish messages function", e );
79 | }
80 | }
81 | }
--------------------------------------------------------------------------------
/spark-notes-matt.txt:
--------------------------------------------------------------------------------
1 | Setup of Spark environment, folders, input data
2 | ----------------------------------------------------
3 |
4 | # Change the next 3 lines
5 | #
6 | export MASTER=ec2-18-197-149-81.eu-central-1.compute.amazonaws.com
7 | export KEYFILE=/home/matt/parking/ec2/flintrock-spark-frankfurt.pem
8 | export CLUSTER=kb
9 |
10 | flintrock run-command ${CLUSTER} 'echo "export HADOOP_HOME=/home/ec2-user/hadoop" >> /home/ec2-user/.bashrc'
11 | flintrock run-command ${CLUSTER} 'echo "export HADOOP_CONF_DIR=/home/ec2-user/hadoop/conf " >> /home/ec2-user/.bashrc'
12 | flintrock run-command ${CLUSTER} 'sudo cp /home/ec2-user/hadoop/lib/native/* /usr/lib/jvm/java-1.8.0-openjdk-1.8.0.191.b12-0.amzn2.x86_64/jre/lib/amd64/'
13 |
14 |
15 | flintrock run-command ${CLUSTER} --master-only 'hdfs dfs -mkdir -p /input /output /tmp /binaries /metadata'
16 |
17 | flintrock run-command ${CLUSTER} --master-only 'wget https://s3.amazonaws.com/mattcasters/customers-noheader-1M.txt -O /tmp/customers-noheader.txt'
18 | flintrock run-command ${CLUSTER} --master-only 'hdfs dfs -put -f /tmp/customers-noheader.txt /input'
19 | flintrock run-command ${CLUSTER} --master-only 'wget https://s3.amazonaws.com/mattcasters/state-data.txt -O /tmp/state-data.txt'
20 | flintrock run-command ${CLUSTER} --master-only 'hdfs dfs -put -f /tmp/state-data.txt /input'
21 |
22 | flintrock run-command ${CLUSTER} --master-only 'mkdir beam'
23 |
24 |
25 | # After running Spark transformation in Spoon
26 | # You'll get a folder filled with goodies...
27 | #
28 | scp -i ${KEYFILE} kettle-beam-fat.jar ec2-user@${MASTER}:beam/
29 | scp -i ${KEYFILE} transformation.ktr ec2-user@${MASTER}:beam/
30 | scp -i ${KEYFILE} metastore.json ec2-user@${MASTER}:beam/
31 |
32 |
33 | # Then you go to the master
34 | #
35 | flintrock login ${CLUSTER}
36 |
37 | # Go to the beam/ folder
38 | #
39 | cd beam/
40 |
41 | # Here we create a file called : submit-command.sh
42 | # TODO: CHANGE THE MASTER
43 | # Copy paste the following
44 |
45 | set -ex
46 |
47 | MASTER=ec2-35-158-121-77.eu-central-1.compute.amazonaws.com
48 |
49 | if [ "$1" != "skip" ]
50 | then
51 | hdfs dfs -put -f kettle-beam-fat.jar /binaries/
52 | hdfs dfs -put -f transformation.ktr /metadata
53 | hdfs dfs -put -f metastore.json /metadata
54 | fi
55 |
56 | spark-submit \
57 | --driver-java-options \
58 | -Djava.library.path=/home/ec2-user/hadoop/lib/native \
59 | --class org.kettle.beam.pipeline.spark.MainSpark \
60 | --master spark://${MASTER}:7077 \
61 | --deploy-mode cluster \
62 | hdfs:///binaries/kettle-beam-fat.jar \
63 | hdfs:///metadata/transformation.ktr \
64 | hdfs:///metadata/metastore.json \
65 | 'Spark' \
66 | org.kettle.beam.steps.io.BeamInputMeta,org.kettle.beam.steps.bq.BeamBQOutputMeta,org.kettle.beam.steps.pubsub.BeamPublishMeta,org.kettle.beam.steps.pubsub.BeamSubscribeMeta,org.kettle.beam.steps.window.BeamTimestampMeta,org.kettle.beam.steps.io.BeamOutputMeta,org.kettle.beam.steps.window.BeamWindowMeta,org.kettle.beam.steps.bq.BeamBQInputMeta \
67 | org.kettle.beam.xp.RunBeamTransExecutionPoint
68 |
69 |
70 |
71 | # Now run this command to start the transformation on Spark
72 | #
73 | sh submit-command.sh
74 |
75 |
76 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/core/transform/BeamInputTransform.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.core.transform;
2 |
3 | import org.apache.beam.sdk.io.Compression;
4 | import org.apache.beam.sdk.io.TextIO;
5 | import org.apache.beam.sdk.metrics.Counter;
6 | import org.apache.beam.sdk.metrics.Metrics;
7 | import org.apache.beam.sdk.transforms.PTransform;
8 | import org.apache.beam.sdk.transforms.ParDo;
9 | import org.apache.beam.sdk.values.PBegin;
10 | import org.apache.beam.sdk.values.PCollection;
11 | import org.kettle.beam.core.BeamKettle;
12 | import org.kettle.beam.core.KettleRow;
13 | import org.kettle.beam.core.fn.StringToKettleFn;
14 | import org.slf4j.Logger;
15 | import org.slf4j.LoggerFactory;
16 |
17 | import javax.annotation.Nullable;
18 | import java.util.List;
19 |
20 | public class BeamInputTransform extends PTransform> {
21 |
22 | // These non-transient privates get serialized to spread across nodes
23 | //
24 | private String stepname;
25 | private String inputLocation;
26 | private String separator;
27 | private String rowMetaJson;
28 | private List stepPluginClasses;
29 | private List xpPluginClasses;
30 |
31 | // Log and count errors.
32 | private static final Logger LOG = LoggerFactory.getLogger( BeamInputTransform.class );
33 | private static final Counter numErrors = Metrics.counter( "main", "BeamInputError" );
34 |
35 | public BeamInputTransform() {
36 | }
37 |
38 | public BeamInputTransform( @Nullable String name, String stepname, String inputLocation, String separator, String rowMetaJson, List stepPluginClasses, List xpPluginClasses ) {
39 | super( name );
40 | this.stepname = stepname;
41 | this.inputLocation = inputLocation;
42 | this.separator = separator;
43 | this.rowMetaJson = rowMetaJson;
44 | this.stepPluginClasses = stepPluginClasses;
45 | this.xpPluginClasses = xpPluginClasses;
46 | }
47 |
48 | @Override public PCollection expand( PBegin input ) {
49 |
50 | try {
51 | // Only initialize once on this node/vm
52 | //
53 | BeamKettle.init(stepPluginClasses, xpPluginClasses);
54 |
55 | // System.out.println("-------------- TextIO.Read from "+inputLocation+" (UNCOMPRESSED)");
56 |
57 | TextIO.Read ioRead = TextIO.read()
58 | .from( inputLocation )
59 | .withCompression( Compression.UNCOMPRESSED )
60 | ;
61 |
62 | StringToKettleFn stringToKettleFn = new StringToKettleFn( stepname, rowMetaJson, separator, stepPluginClasses, xpPluginClasses );
63 |
64 | PCollection output = input
65 |
66 | // We read a bunch of Strings, one per line basically
67 | //
68 | .apply( stepname + " READ FILE", ioRead )
69 |
70 | // We need to transform these lines into Kettle fields
71 | //
72 | .apply( stepname, ParDo.of( stringToKettleFn ) );
73 |
74 | return output;
75 |
76 | } catch ( Exception e ) {
77 | numErrors.inc();
78 | LOG.error( "Error in beam input transform", e );
79 | throw new RuntimeException( "Error in beam input transform", e );
80 | }
81 |
82 | }
83 |
84 |
85 | }
86 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/pipeline/handler/BeamBigQueryOutputStepHandler.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.pipeline.handler;
2 |
3 | import org.apache.beam.sdk.Pipeline;
4 | import org.apache.beam.sdk.values.PCollection;
5 | import org.apache.commons.lang.StringUtils;
6 | import org.kettle.beam.core.KettleRow;
7 | import org.kettle.beam.core.transform.BeamBQOutputTransform;
8 | import org.kettle.beam.core.transform.BeamOutputTransform;
9 | import org.kettle.beam.core.util.JsonRowMeta;
10 | import org.kettle.beam.metastore.BeamJobConfig;
11 | import org.kettle.beam.metastore.FieldDefinition;
12 | import org.kettle.beam.metastore.FileDefinition;
13 | import org.kettle.beam.steps.bq.BeamBQOutputMeta;
14 | import org.kettle.beam.steps.io.BeamOutputMeta;
15 | import org.pentaho.di.core.exception.KettleException;
16 | import org.pentaho.di.core.exception.KettleStepException;
17 | import org.pentaho.di.core.logging.LogChannelInterface;
18 | import org.pentaho.di.core.row.RowMetaInterface;
19 | import org.pentaho.di.core.row.ValueMetaInterface;
20 | import org.pentaho.di.trans.TransMeta;
21 | import org.pentaho.di.trans.step.StepMeta;
22 | import org.pentaho.metastore.api.IMetaStore;
23 |
24 | import java.util.List;
25 | import java.util.Map;
26 |
27 | public class BeamBigQueryOutputStepHandler extends BeamBaseStepHandler implements BeamStepHandler {
28 |
29 | public BeamBigQueryOutputStepHandler( BeamJobConfig beamJobConfig, IMetaStore metaStore, TransMeta transMeta, List stepPluginClasses, List xpPluginClasses ) {
30 | super( beamJobConfig, false, true, metaStore, transMeta, stepPluginClasses, xpPluginClasses );
31 | }
32 |
33 | @Override public void handleStep( LogChannelInterface log, StepMeta beamOutputStepMeta, Map> stepCollectionMap,
34 | Pipeline pipeline, RowMetaInterface rowMeta, List previousSteps,
35 | PCollection input ) throws KettleException {
36 |
37 | BeamBQOutputMeta beamOutputMeta = (BeamBQOutputMeta) beamOutputStepMeta.getStepMetaInterface();
38 |
39 | BeamBQOutputTransform beamOutputTransform = new BeamBQOutputTransform(
40 | beamOutputStepMeta.getName(),
41 | transMeta.environmentSubstitute( beamOutputMeta.getProjectId() ),
42 | transMeta.environmentSubstitute( beamOutputMeta.getDatasetId() ),
43 | transMeta.environmentSubstitute( beamOutputMeta.getTableId() ),
44 | beamOutputMeta.isCreatingIfNeeded(),
45 | beamOutputMeta.isTruncatingTable(),
46 | beamOutputMeta.isFailingIfNotEmpty(),
47 | JsonRowMeta.toJson(rowMeta),
48 | stepPluginClasses,
49 | xpPluginClasses
50 | );
51 |
52 | // Which step do we apply this transform to?
53 | // Ignore info hops until we figure that out.
54 | //
55 | if ( previousSteps.size() > 1 ) {
56 | throw new KettleException( "Combining data from multiple steps is not supported yet!" );
57 | }
58 | StepMeta previousStep = previousSteps.get( 0 );
59 |
60 | // No need to store this, it's PDone.
61 | //
62 | input.apply( beamOutputTransform );
63 | log.logBasic( "Handled step (BQ OUTPUT) : " + beamOutputStepMeta.getName() + ", gets data from " + previousStep.getName() );
64 | }
65 | }
66 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/pipeline/handler/BeamBaseStepHandler.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.pipeline.handler;
2 |
3 | import org.kettle.beam.metastore.BeamJobConfig;
4 | import org.pentaho.di.trans.TransMeta;
5 | import org.pentaho.metastore.api.IMetaStore;
6 |
7 | import java.util.List;
8 |
9 | public class BeamBaseStepHandler {
10 |
11 | protected IMetaStore metaStore;
12 | protected TransMeta transMeta;
13 | protected List stepPluginClasses;
14 | protected List xpPluginClasses;
15 | protected boolean input;
16 | protected boolean output;
17 | protected BeamJobConfig beamJobConfig;
18 |
19 | public BeamBaseStepHandler( BeamJobConfig beamJobConfig, boolean input, boolean output, IMetaStore metaStore, TransMeta transMeta, List stepPluginClasses, List xpPluginClasses ) {
20 | this.beamJobConfig = beamJobConfig;
21 | this.input = input;
22 | this.output = output;
23 | this.metaStore = metaStore;
24 | this.transMeta = transMeta;
25 | this.stepPluginClasses = stepPluginClasses;
26 | this.xpPluginClasses = xpPluginClasses;
27 | }
28 |
29 | /**
30 | * Gets metaStore
31 | *
32 | * @return value of metaStore
33 | */
34 | public IMetaStore getMetaStore() {
35 | return metaStore;
36 | }
37 |
38 | /**
39 | * @param metaStore The metaStore to set
40 | */
41 | public void setMetaStore( IMetaStore metaStore ) {
42 | this.metaStore = metaStore;
43 | }
44 |
45 | /**
46 | * Gets transMeta
47 | *
48 | * @return value of transMeta
49 | */
50 | public TransMeta getTransMeta() {
51 | return transMeta;
52 | }
53 |
54 | /**
55 | * @param transMeta The transMeta to set
56 | */
57 | public void setTransMeta( TransMeta transMeta ) {
58 | this.transMeta = transMeta;
59 | }
60 |
61 | /**
62 | * Gets stepPluginClasses
63 | *
64 | * @return value of stepPluginClasses
65 | */
66 | public List getStepPluginClasses() {
67 | return stepPluginClasses;
68 | }
69 |
70 | /**
71 | * @param stepPluginClasses The stepPluginClasses to set
72 | */
73 | public void setStepPluginClasses( List stepPluginClasses ) {
74 | this.stepPluginClasses = stepPluginClasses;
75 | }
76 |
77 | /**
78 | * Gets xpPluginClasses
79 | *
80 | * @return value of xpPluginClasses
81 | */
82 | public List getXpPluginClasses() {
83 | return xpPluginClasses;
84 | }
85 |
86 | /**
87 | * @param xpPluginClasses The xpPluginClasses to set
88 | */
89 | public void setXpPluginClasses( List xpPluginClasses ) {
90 | this.xpPluginClasses = xpPluginClasses;
91 | }
92 |
93 | /**
94 | * Gets input
95 | *
96 | * @return value of input
97 | */
98 | public boolean isInput() {
99 | return input;
100 | }
101 |
102 | /**
103 | * @param input The input to set
104 | */
105 | public void setInput( boolean input ) {
106 | this.input = input;
107 | }
108 |
109 | /**
110 | * Gets output
111 | *
112 | * @return value of output
113 | */
114 | public boolean isOutput() {
115 | return output;
116 | }
117 |
118 | /**
119 | * @param output The output to set
120 | */
121 | public void setOutput( boolean output ) {
122 | this.output = output;
123 | }
124 | }
125 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/metastore/FieldDefinition.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.metastore;
2 |
3 | import org.pentaho.di.core.exception.KettlePluginException;
4 | import org.pentaho.di.core.row.ValueMetaInterface;
5 | import org.pentaho.di.core.row.value.ValueMetaFactory;
6 | import org.pentaho.metastore.persist.MetaStoreAttribute;
7 |
8 | import java.io.Serializable;
9 |
10 | public class FieldDefinition implements Serializable {
11 |
12 | @MetaStoreAttribute
13 | private String name;
14 |
15 | @MetaStoreAttribute
16 | private String kettleType;
17 |
18 | @MetaStoreAttribute
19 | private int length;
20 |
21 | @MetaStoreAttribute
22 | private int precision;
23 |
24 | @MetaStoreAttribute
25 | private String formatMask;
26 |
27 | public FieldDefinition( ) {
28 | }
29 |
30 | public FieldDefinition( String name, String kettleType, int length, int precision ) {
31 | this.name = name;
32 | this.kettleType = kettleType;
33 | this.length = length;
34 | this.precision = precision;
35 | }
36 |
37 | public FieldDefinition( String name, String kettleType, int length, int precision, String formatMask ) {
38 | this.name = name;
39 | this.kettleType = kettleType;
40 | this.length = length;
41 | this.precision = precision;
42 | this.formatMask = formatMask;
43 | }
44 |
45 | public ValueMetaInterface getValueMeta() throws KettlePluginException {
46 | int type = ValueMetaFactory.getIdForValueMeta( kettleType );
47 | ValueMetaInterface valueMeta = ValueMetaFactory.createValueMeta( name, type, length, precision );
48 | valueMeta.setConversionMask( formatMask );
49 | return valueMeta;
50 | }
51 |
52 | /**
53 | * Gets name
54 | *
55 | * @return value of name
56 | */
57 | public String getName() {
58 | return name;
59 | }
60 |
61 | /**
62 | * @param name The name to set
63 | */
64 | public void setName( String name ) {
65 | this.name = name;
66 | }
67 |
68 | /**
69 | * Gets kettleType
70 | *
71 | * @return value of kettleType
72 | */
73 | public String getKettleType() {
74 | return kettleType;
75 | }
76 |
77 | /**
78 | * @param kettleType The kettleType to set
79 | */
80 | public void setKettleType( String kettleType ) {
81 | this.kettleType = kettleType;
82 | }
83 |
84 | /**
85 | * Gets length
86 | *
87 | * @return value of length
88 | */
89 | public int getLength() {
90 | return length;
91 | }
92 |
93 | /**
94 | * @param length The length to set
95 | */
96 | public void setLength( int length ) {
97 | this.length = length;
98 | }
99 |
100 | /**
101 | * Gets precision
102 | *
103 | * @return value of precision
104 | */
105 | public int getPrecision() {
106 | return precision;
107 | }
108 |
109 | /**
110 | * @param precision The precision to set
111 | */
112 | public void setPrecision( int precision ) {
113 | this.precision = precision;
114 | }
115 |
116 | /**
117 | * Gets formatMask
118 | *
119 | * @return value of formatMask
120 | */
121 | public String getFormatMask() {
122 | return formatMask;
123 | }
124 |
125 | /**
126 | * @param formatMask The formatMask to set
127 | */
128 | public void setFormatMask( String formatMask ) {
129 | this.formatMask = formatMask;
130 | }
131 | }
132 |
--------------------------------------------------------------------------------
/src/test/java/org/kettle/beam/transform/PipelineTestBase.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.transform;
2 |
3 | import org.apache.beam.runners.direct.DirectRunner;
4 | import org.apache.beam.sdk.Pipeline;
5 | import org.apache.beam.sdk.PipelineResult;
6 | import org.apache.beam.sdk.metrics.MetricQueryResults;
7 | import org.apache.beam.sdk.metrics.MetricResult;
8 | import org.apache.beam.sdk.metrics.MetricResults;
9 | import org.apache.beam.sdk.metrics.MetricsFilter;
10 | import org.apache.beam.sdk.options.PipelineOptions;
11 | import org.apache.beam.sdk.options.PipelineOptionsFactory;
12 | import org.apache.commons.io.FileUtils;
13 | import org.junit.Before;
14 | import org.junit.Ignore;
15 | import org.kettle.beam.core.BeamDefaults;
16 | import org.kettle.beam.core.BeamKettle;
17 | import org.kettle.beam.metastore.BeamJobConfig;
18 | import org.kettle.beam.metastore.RunnerType;
19 | import org.kettle.beam.pipeline.TransMetaPipelineConverter;
20 | import org.kettle.beam.util.BeamConst;
21 | import org.pentaho.di.trans.TransMeta;
22 | import org.pentaho.metastore.api.IMetaStore;
23 | import org.pentaho.metastore.stores.memory.MemoryMetaStore;
24 |
25 | import java.io.File;
26 | import java.util.ArrayList;
27 |
28 | public class PipelineTestBase {
29 |
30 | protected IMetaStore metaStore;
31 |
32 | @Before
33 | public void setUp() throws Exception {
34 | BeamKettle.init( new ArrayList<>(), new ArrayList<>() );
35 |
36 | metaStore = new MemoryMetaStore();
37 |
38 | File inputFolder = new File( "/tmp/customers/io" );
39 | inputFolder.mkdirs();
40 | File outputFolder = new File( "/tmp/customers/output" );
41 | outputFolder.mkdirs();
42 | File tmpFolder = new File( "/tmp/customers/tmp" );
43 | tmpFolder.mkdirs();
44 |
45 | FileUtils.copyFile( new File( "src/test/resources/customers/customers-100.txt" ), new File( "/tmp/customers/io/customers-100.txt" ) );
46 | FileUtils.copyFile( new File( "src/test/resources/customers/state-data.txt" ), new File( "/tmp/customers/io/state-data.txt" ) );
47 | }
48 |
49 |
50 | @Ignore
51 | public void createRunPipeline( TransMeta transMeta ) throws Exception {
52 |
53 | /*
54 | FileOutputStream fos = new FileOutputStream( "/tmp/"+transMeta.getName()+".ktr" );
55 | fos.write( transMeta.getXML().getBytes() );
56 | fos.close();
57 | */
58 |
59 | PipelineOptions pipelineOptions = PipelineOptionsFactory.create();
60 |
61 | pipelineOptions.setJobName( transMeta.getName() );
62 | pipelineOptions.setUserAgent( BeamConst.STRING_KETTLE_BEAM );
63 |
64 | BeamJobConfig jobConfig = new BeamJobConfig();
65 | jobConfig.setName("Direct runner test");
66 | jobConfig.setRunnerTypeName( RunnerType.Direct.name() );
67 |
68 | // No extra plugins to load : null option
69 | TransMetaPipelineConverter converter = new TransMetaPipelineConverter( transMeta, metaStore, (String) null, jobConfig );
70 | Pipeline pipeline = converter.createPipeline( pipelineOptions );
71 |
72 | PipelineResult pipelineResult = pipeline.run();
73 | pipelineResult.waitUntilFinish();
74 |
75 | MetricResults metricResults = pipelineResult.metrics();
76 |
77 | MetricQueryResults allResults = metricResults.queryMetrics( MetricsFilter.builder().build() );
78 | for ( MetricResult result : allResults.getCounters() ) {
79 | System.out.println( "Name: " + result.getName() + " Attempted: " + result.getAttempted() );
80 | }
81 | }
82 | }
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/metastore/FileDefinition.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.metastore;
2 |
3 | import org.pentaho.di.core.exception.KettlePluginException;
4 | import org.pentaho.di.core.row.RowMeta;
5 | import org.pentaho.di.core.row.RowMetaInterface;
6 | import org.pentaho.metastore.persist.MetaStoreAttribute;
7 | import org.pentaho.metastore.persist.MetaStoreElementType;
8 |
9 | import java.io.Serializable;
10 | import java.util.ArrayList;
11 | import java.util.List;
12 |
13 | @MetaStoreElementType(
14 | name = "File Definition",
15 | description = "Describes a file layout"
16 | )
17 | public class FileDefinition implements Serializable {
18 |
19 | private String name;
20 |
21 | @MetaStoreAttribute
22 | private String description;
23 |
24 | @MetaStoreAttribute
25 | private List fieldDefinitions;
26 |
27 | @MetaStoreAttribute
28 | private String separator;
29 |
30 | @MetaStoreAttribute
31 | private String enclosure;
32 |
33 | public FileDefinition() {
34 | fieldDefinitions = new ArrayList<>();
35 | }
36 |
37 | public FileDefinition( String name, String description, List fieldDefinitions, String separator, String enclosure ) {
38 | this.name = name;
39 | this.description = description;
40 | this.fieldDefinitions = fieldDefinitions;
41 | this.separator = separator;
42 | this.enclosure = enclosure;
43 | }
44 |
45 | public RowMetaInterface getRowMeta() throws KettlePluginException {
46 | RowMetaInterface rowMeta = new RowMeta();
47 |
48 | for (FieldDefinition fieldDefinition : fieldDefinitions) {
49 | rowMeta.addValueMeta( fieldDefinition.getValueMeta() );
50 | }
51 |
52 | return rowMeta;
53 | }
54 |
55 | /**
56 | * Gets name
57 | *
58 | * @return value of name
59 | */
60 | public String getName() {
61 | return name;
62 | }
63 |
64 | /**
65 | * @param name The name to set
66 | */
67 | public void setName( String name ) {
68 | this.name = name;
69 | }
70 |
71 | /**
72 | * Gets description
73 | *
74 | * @return value of description
75 | */
76 | public String getDescription() {
77 | return description;
78 | }
79 |
80 | /**
81 | * @param description The description to set
82 | */
83 | public void setDescription( String description ) {
84 | this.description = description;
85 | }
86 |
87 | /**
88 | * Gets fieldDefinitions
89 | *
90 | * @return value of fieldDefinitions
91 | */
92 | public List getFieldDefinitions() {
93 | return fieldDefinitions;
94 | }
95 |
96 | /**
97 | * @param fieldDefinitions The fieldDefinitions to set
98 | */
99 | public void setFieldDefinitions( List fieldDefinitions ) {
100 | this.fieldDefinitions = fieldDefinitions;
101 | }
102 |
103 | /**
104 | * Gets separator
105 | *
106 | * @return value of separator
107 | */
108 | public String getSeparator() {
109 | return separator;
110 | }
111 |
112 | /**
113 | * @param separator The separator to set
114 | */
115 | public void setSeparator( String separator ) {
116 | this.separator = separator;
117 | }
118 |
119 | /**
120 | * Gets enclosure
121 | *
122 | * @return value of enclosure
123 | */
124 | public String getEnclosure() {
125 | return enclosure;
126 | }
127 |
128 | /**
129 | * @param enclosure The enclosure to set
130 | */
131 | public void setEnclosure( String enclosure ) {
132 | this.enclosure = enclosure;
133 | }
134 | }
135 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/core/transform/BeamBQInputTransform.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.core.transform;
2 |
3 | import com.google.api.services.bigquery.model.TableReference;
4 | import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
5 | import org.apache.beam.sdk.metrics.Counter;
6 | import org.apache.beam.sdk.metrics.Metrics;
7 | import org.apache.beam.sdk.transforms.PTransform;
8 | import org.apache.beam.sdk.values.PBegin;
9 | import org.apache.beam.sdk.values.PCollection;
10 | import org.apache.commons.lang.StringUtils;
11 | import org.kettle.beam.core.BeamKettle;
12 | import org.kettle.beam.core.KettleRow;
13 | import org.kettle.beam.core.fn.BQSchemaAndRecordToKettleFn;
14 | import org.pentaho.di.core.row.RowMetaInterface;
15 | import org.slf4j.Logger;
16 | import org.slf4j.LoggerFactory;
17 |
18 | import javax.annotation.Nullable;
19 | import java.util.List;
20 |
21 | public class BeamBQInputTransform extends PTransform> {
22 |
23 | // These non-transient privates get serialized to spread across nodes
24 | //
25 | private String stepname;
26 | private String projectId;
27 | private String datasetId;
28 | private String tableId;
29 | private String query;
30 | private String rowMetaJson;
31 | private List stepPluginClasses;
32 | private List xpPluginClasses;
33 |
34 | // Log and count errors.
35 | private static final Logger LOG = LoggerFactory.getLogger( BeamBQInputTransform.class );
36 | private static final Counter numErrors = Metrics.counter( "main", "BeamBQInputError" );
37 |
38 | private transient RowMetaInterface rowMeta;
39 |
40 | public BeamBQInputTransform() {
41 | }
42 |
43 | public BeamBQInputTransform( @Nullable String name, String stepname, String projectId, String datasetId, String tableId, String query, String rowMetaJson, List stepPluginClasses, List xpPluginClasses ) {
44 | super( name );
45 | this.stepname = stepname;
46 | this.projectId = projectId;
47 | this.datasetId = datasetId;
48 | this.tableId = tableId;
49 | this.query = query;
50 | this.rowMetaJson = rowMetaJson;
51 | this.stepPluginClasses = stepPluginClasses;
52 | this.xpPluginClasses = xpPluginClasses;
53 | }
54 |
55 | @Override public PCollection expand( PBegin input ) {
56 | try {
57 | // Only initialize once on this node/vm
58 | //
59 | BeamKettle.init(stepPluginClasses, xpPluginClasses);
60 |
61 | // Function to convert from Avro to Kettle rows
62 | //
63 | BQSchemaAndRecordToKettleFn toKettleFn = new BQSchemaAndRecordToKettleFn( stepname, rowMetaJson, stepPluginClasses, xpPluginClasses );
64 |
65 | TableReference tableReference = new TableReference();
66 | if (StringUtils.isNotEmpty( projectId )) {
67 | tableReference.setProjectId( projectId );
68 | }
69 | tableReference.setDatasetId( datasetId );
70 | tableReference.setTableId( tableId );
71 |
72 | BigQueryIO.TypedRead bqTypedRead;
73 |
74 | if (StringUtils.isEmpty( query )) {
75 | bqTypedRead = BigQueryIO
76 | .read( toKettleFn )
77 | .from( tableReference )
78 | ;
79 | } else {
80 | bqTypedRead = BigQueryIO
81 | .read( toKettleFn )
82 | .fromQuery( query )
83 | ;
84 | }
85 |
86 | // Apply the function
87 | //
88 | PCollection output = input.apply( bqTypedRead );
89 |
90 | return output;
91 |
92 | } catch ( Exception e ) {
93 | numErrors.inc();
94 | LOG.error( "Error in beam input transform", e );
95 | throw new RuntimeException( "Error in beam input transform", e );
96 | }
97 | }
98 | }
99 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/pipeline/handler/BeamKafkaInputStepHandler.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.pipeline.handler;
2 |
3 | import org.apache.beam.sdk.Pipeline;
4 | import org.apache.beam.sdk.values.PCollection;
5 | import org.kettle.beam.core.KettleRow;
6 | import org.kettle.beam.core.transform.BeamKafkaInputTransform;
7 | import org.kettle.beam.core.util.JsonRowMeta;
8 | import org.kettle.beam.metastore.BeamJobConfig;
9 | import org.kettle.beam.steps.kafka.BeamConsumeMeta;
10 | import org.kettle.beam.steps.kafka.ConfigOption;
11 | import org.pentaho.di.core.exception.KettleException;
12 | import org.pentaho.di.core.logging.LogChannelInterface;
13 | import org.pentaho.di.core.row.RowMeta;
14 | import org.pentaho.di.core.row.RowMetaInterface;
15 | import org.pentaho.di.trans.TransMeta;
16 | import org.pentaho.di.trans.step.StepMeta;
17 | import org.pentaho.metastore.api.IMetaStore;
18 |
19 | import java.util.List;
20 | import java.util.Map;
21 |
22 | public class BeamKafkaInputStepHandler extends BeamBaseStepHandler implements BeamStepHandler {
23 |
24 | public BeamKafkaInputStepHandler( BeamJobConfig beamJobConfig, IMetaStore metaStore, TransMeta transMeta, List stepPluginClasses, List xpPluginClasses ) {
25 | super( beamJobConfig, true, false, metaStore, transMeta, stepPluginClasses, xpPluginClasses );
26 | }
27 |
28 | @Override public void handleStep( LogChannelInterface log, StepMeta stepMeta, Map> stepCollectionMap,
29 | Pipeline pipeline, RowMetaInterface rowMeta, List previousSteps,
30 | PCollection input ) throws KettleException {
31 |
32 | // Input handling
33 | //
34 | BeamConsumeMeta beamConsumeMeta = (BeamConsumeMeta) stepMeta.getStepMetaInterface();
35 |
36 | // Output rows (fields selection)
37 | //
38 | RowMetaInterface outputRowMeta = new RowMeta();
39 | beamConsumeMeta.getFields( outputRowMeta, stepMeta.getName(), null, null, transMeta, null, null );
40 |
41 | String[] parameters = new String[beamConsumeMeta.getConfigOptions().size()];
42 | String[] values = new String[beamConsumeMeta.getConfigOptions().size()];
43 | String[] types = new String[beamConsumeMeta.getConfigOptions().size()];
44 | for (int i=0;i afterInput = pipeline.apply( beamInputTransform );
70 | stepCollectionMap.put( stepMeta.getName(), afterInput );
71 | log.logBasic( "Handled step (KAFKA INPUT) : " + stepMeta.getName() );
72 | }
73 | }
74 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/carte/RegisterBeamMetrics.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.carte;
2 |
3 | import java.io.BufferedReader;
4 | import java.io.IOException;
5 | import java.io.PrintWriter;
6 | import java.util.Date;
7 |
8 | import javax.servlet.ServletException;
9 | import javax.servlet.http.HttpServletRequest;
10 | import javax.servlet.http.HttpServletResponse;
11 |
12 | import org.pentaho.di.core.Const;
13 | import org.pentaho.di.core.annotations.CarteServlet;
14 | import org.pentaho.di.core.xml.XMLHandler;
15 | import org.pentaho.di.www.BaseHttpServlet;
16 | import org.pentaho.di.www.CartePluginInterface;
17 | import org.pentaho.di.www.SlaveServerTransStatus;
18 | import org.pentaho.di.www.WebResult;
19 |
20 | @CarteServlet(
21 | id="registerBeamMetrics",
22 | name="Register Beam Metrics",
23 | description="Captures Apache Beam Metrics regarding a running transformation"
24 | )
25 | public class RegisterBeamMetrics extends BaseHttpServlet implements CartePluginInterface {
26 |
27 | private static final long serialVersionUIDL = 348324987293472947L;
28 |
29 | public static final String CONTEXT_PATH = "/kettle/registerBeamMetrics";
30 |
31 | public RegisterBeamMetrics() {
32 | }
33 |
34 | public String toString() {
35 | return "Register Beam Metrics";
36 | }
37 |
38 | public String getService() {
39 | return CONTEXT_PATH + " (" + toString() + ")";
40 | }
41 |
42 | public String getContextPath() {
43 | return CONTEXT_PATH;
44 | }
45 |
46 | @Override
47 | public void doGet(HttpServletRequest request, HttpServletResponse response) throws ServletException, IOException {
48 |
49 | if (isJettyMode() && !request.getRequestURI().startsWith(CONTEXT_PATH))
50 | return;
51 |
52 | if (log.isDebug()) {
53 | logDebug("Registration of Apache Beam Metrics");
54 | }
55 |
56 | // The Object ID
57 | //
58 | String carteObjectId = request.getParameter("id"); // the carte object id
59 |
60 | // Transformation name
61 | //
62 | String trans = request.getParameter("trans"); // the name of the transformation
63 |
64 | // Internal Job ID
65 | //
66 | String internalJobId = request.getParameter("internalJobId"); // the Id of the Spark/Flink job
67 |
68 | // Update date
69 | //
70 | Date updateDate = new Date();
71 |
72 | PrintWriter out = response.getWriter();
73 | BufferedReader in = request.getReader();
74 |
75 | WebResult webResult = new WebResult(WebResult.STRING_OK, "registration success", "");
76 |
77 | try {
78 |
79 | // First read the complete SlaveServerTransStatus object XML in memory from the request
80 | //
81 | StringBuilder xml = new StringBuilder(request.getContentLength());
82 | int c;
83 | while ((c = in.read()) != -1) {
84 | xml.append((char) c);
85 | }
86 |
87 | SlaveServerTransStatus transStatus = SlaveServerTransStatus.fromXML(xml.toString());
88 |
89 | MetricsRegistrationQueue registry = MetricsRegistrationQueue.getInstance();
90 | BeamMetricsEntry entry = new BeamMetricsEntry(carteObjectId, trans, internalJobId, updateDate, transStatus);
91 | registry.addNodeRegistryEntry(entry);
92 |
93 | response.setContentType("text/xml");
94 | response.setStatus(HttpServletResponse.SC_OK);
95 | response.setCharacterEncoding(Const.XML_ENCODING);
96 | out.println(XMLHandler.getXMLHeader());
97 | } catch(Exception e) {
98 | webResult.setResult(WebResult.STRING_ERROR);
99 | webResult.setMessage(Const.getStackTracker(e));
100 | }
101 | out.println(webResult.getXML());
102 | }
103 | }
--------------------------------------------------------------------------------
/src/main/resources/beam-bq-input.svg:
--------------------------------------------------------------------------------
1 |
2 |
105 |
--------------------------------------------------------------------------------
/src/main/resources/beam-bq-output.svg:
--------------------------------------------------------------------------------
1 |
2 |
105 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/core/fn/KettleToBQTableRowFn.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.core.fn;
2 |
3 | import com.google.api.services.bigquery.model.TableRow;
4 | import org.apache.beam.sdk.metrics.Counter;
5 | import org.apache.beam.sdk.metrics.Metrics;
6 | import org.apache.beam.sdk.transforms.SerializableFunction;
7 | import org.kettle.beam.core.BeamKettle;
8 | import org.kettle.beam.core.KettleRow;
9 | import org.kettle.beam.core.util.JsonRowMeta;
10 | import org.pentaho.di.core.row.RowMetaInterface;
11 | import org.pentaho.di.core.row.ValueMetaInterface;
12 | import org.slf4j.Logger;
13 | import org.slf4j.LoggerFactory;
14 |
15 | import java.text.SimpleDateFormat;
16 | import java.util.Date;
17 | import java.util.List;
18 |
19 | public class KettleToBQTableRowFn implements SerializableFunction {
20 |
21 | private String counterName;
22 | private String rowMetaJson;
23 | private List stepPluginClasses;
24 | private List xpPluginClasses;
25 |
26 | private transient RowMetaInterface rowMeta;
27 | private transient Counter initCounter;
28 | private transient Counter readCounter;
29 | private transient Counter outputCounter;
30 | private transient Counter errorCounter;
31 |
32 | private transient SimpleDateFormat simpleDateFormat;
33 |
34 | // Log and count parse errors.
35 | private static final Logger LOG = LoggerFactory.getLogger( KettleToBQTableRowFn.class );
36 |
37 | public KettleToBQTableRowFn( String counterName, String rowMetaJson, List stepPluginClasses, List xpPluginClasses ) {
38 | this.counterName = counterName;
39 | this.rowMetaJson = rowMetaJson;
40 | this.stepPluginClasses = stepPluginClasses;
41 | this.xpPluginClasses = xpPluginClasses;
42 | }
43 |
44 | @Override public TableRow apply( KettleRow inputRow ) {
45 |
46 | try {
47 | if ( rowMeta == null ) {
48 | readCounter = Metrics.counter( "read", counterName );
49 | outputCounter = Metrics.counter( "output", counterName );
50 | errorCounter = Metrics.counter( "error", counterName );
51 |
52 | // Initialize Kettle Beam
53 | //
54 | BeamKettle.init( stepPluginClasses, xpPluginClasses );
55 | rowMeta = JsonRowMeta.fromJson( rowMetaJson );
56 |
57 | simpleDateFormat = new SimpleDateFormat( "yyyy-MM-dd HH:mm:ss.SSS" );
58 | Metrics.counter( "init", counterName ).inc();
59 | }
60 |
61 | readCounter.inc();
62 |
63 | TableRow tableRow = new TableRow();
64 | for (int i=0;i {
17 |
18 | private String counterName;
19 | private String outputLocation;
20 | private String separator;
21 | private String enclosure;
22 | private String rowMetaJson;
23 | private List stepPluginClasses;
24 | private List xpPluginClasses;
25 |
26 | private transient RowMetaInterface rowMeta;
27 | private transient Counter initCounter;
28 | private transient Counter readCounter;
29 | private transient Counter outputCounter;
30 | private transient Counter errorCounter;
31 |
32 | // Log and count parse errors.
33 | private static final Logger LOG = LoggerFactory.getLogger( KettleToStringFn.class );
34 |
35 | public KettleToStringFn( String counterName, String outputLocation, String separator, String enclosure, String rowMetaJson, List stepPluginClasses, List xpPluginClasses ) {
36 | this.counterName = counterName;
37 | this.outputLocation = outputLocation;
38 | this.separator = separator;
39 | this.enclosure = enclosure;
40 | this.rowMetaJson = rowMetaJson;
41 | this.stepPluginClasses = stepPluginClasses;
42 | this.xpPluginClasses = xpPluginClasses;
43 | }
44 |
45 | @Setup
46 | public void setUp() {
47 | try {
48 | readCounter = Metrics.counter( "read", counterName );
49 | outputCounter = Metrics.counter( "output", counterName );
50 | errorCounter = Metrics.counter( "error", counterName );
51 |
52 | // Initialize Kettle Beam
53 | //
54 | BeamKettle.init( stepPluginClasses, xpPluginClasses );
55 | rowMeta = JsonRowMeta.fromJson( rowMetaJson );
56 |
57 | Metrics.counter( "init", counterName ).inc();
58 | } catch ( Exception e ) {
59 | errorCounter.inc();
60 | LOG.info( "Parse error on setup of Kettle data to string lines : " + e.getMessage() );
61 | throw new RuntimeException( "Error on setup of converting Kettle data to string lines", e );
62 | }
63 | }
64 |
65 | @ProcessElement
66 | public void processElement( ProcessContext processContext ) {
67 |
68 | try {
69 |
70 | KettleRow inputRow = processContext.element();
71 | readCounter.inc();
72 |
73 | // Just a quick and dirty output for now...
74 | // TODO: refine with multiple output formats, Avro, Parquet, ...
75 | //
76 | StringBuffer line = new StringBuffer();
77 |
78 | for ( int i = 0; i < rowMeta.size(); i++ ) {
79 |
80 | if ( i > 0 ) {
81 | line.append( separator );
82 | }
83 |
84 | String valueString = rowMeta.getString( inputRow.getRow(), i );
85 |
86 | if ( valueString != null ) {
87 | boolean enclose = false;
88 | if ( StringUtils.isNotEmpty( enclosure ) ) {
89 | enclose = valueString.contains( enclosure );
90 | }
91 | if ( enclose ) {
92 | line.append( enclosure );
93 | }
94 | line.append( valueString );
95 | if ( enclose ) {
96 | line.append( enclosure );
97 | }
98 | }
99 | }
100 |
101 | // Pass the row to the process context
102 | //
103 | processContext.output( line.toString() );
104 | outputCounter.inc();
105 |
106 | } catch ( Exception e ) {
107 | errorCounter.inc();
108 | LOG.info( "Parse error on " + processContext.element() + ", " + e.getMessage() );
109 | throw new RuntimeException( "Error converting Kettle data to string lines", e );
110 | }
111 | }
112 |
113 |
114 | }
115 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/core/transform/BeamKafkaOutputTransform.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.core.transform;
2 |
3 | import org.apache.beam.sdk.io.kafka.KafkaIO;
4 | import org.apache.beam.sdk.metrics.Counter;
5 | import org.apache.beam.sdk.metrics.Metrics;
6 | import org.apache.beam.sdk.transforms.PTransform;
7 | import org.apache.beam.sdk.transforms.ParDo;
8 | import org.apache.beam.sdk.values.KV;
9 | import org.apache.beam.sdk.values.PCollection;
10 | import org.apache.beam.sdk.values.PDone;
11 | import org.apache.kafka.common.serialization.StringSerializer;
12 | import org.kettle.beam.core.BeamKettle;
13 | import org.kettle.beam.core.KettleRow;
14 | import org.kettle.beam.core.fn.KettleRowToKVStringStringFn;
15 | import org.kettle.beam.core.util.JsonRowMeta;
16 | import org.pentaho.di.core.exception.KettleException;
17 | import org.pentaho.di.core.row.RowMetaInterface;
18 | import org.slf4j.Logger;
19 | import org.slf4j.LoggerFactory;
20 |
21 | import java.util.List;
22 |
23 | public class BeamKafkaOutputTransform extends PTransform, PDone> {
24 |
25 | // These non-transient privates get serialized to spread across nodes
26 | //
27 | private String stepname;
28 | private String bootstrapServers;
29 | private String topic;
30 | private String keyField;
31 | private String messageField;
32 | private String rowMetaJson;
33 | private List stepPluginClasses;
34 | private List xpPluginClasses;
35 |
36 | // Log and count errors.
37 | private static final Logger LOG = LoggerFactory.getLogger( BeamKafkaOutputTransform.class );
38 | private static final Counter numErrors = Metrics.counter( "main", "BeamKafkaOutputError" );
39 |
40 | public BeamKafkaOutputTransform() {
41 | }
42 |
43 | public BeamKafkaOutputTransform( String stepname, String bootstrapServers, String topic, String keyField, String messageField, String rowMetaJson, List stepPluginClasses, List xpPluginClasses ) {
44 | this.stepname = stepname;
45 | this.bootstrapServers = bootstrapServers;
46 | this.topic = topic;
47 | this.keyField = keyField;
48 | this.messageField = messageField;
49 | this.rowMetaJson = rowMetaJson;
50 | this.stepPluginClasses = stepPluginClasses;
51 | this.xpPluginClasses = xpPluginClasses;
52 | }
53 |
54 | @Override public PDone expand( PCollection input ) {
55 |
56 | try {
57 | // Only initialize once on this node/vm
58 | //
59 | BeamKettle.init( stepPluginClasses, xpPluginClasses );
60 |
61 | // Inflate the metadata on the node where this is running...
62 | //
63 | RowMetaInterface rowMeta = JsonRowMeta.fromJson( rowMetaJson );
64 |
65 | int keyIndex = rowMeta.indexOfValue( keyField );
66 | if (keyIndex<0) {
67 | throw new KettleException( "Unable to find key field "+keyField+" in input row: "+rowMeta.toString() );
68 | }
69 | int messageIndex = rowMeta.indexOfValue( messageField );
70 | if (messageIndex<0) {
71 | throw new KettleException( "Unable to find message field "+messageField+" in input row: "+rowMeta.toString() );
72 | }
73 |
74 | // First convert the input stream of KettleRows to KV for the keys and messages
75 | //
76 | KettleRowToKVStringStringFn kettleRowToKVStringStringFn = new KettleRowToKVStringStringFn( stepname, keyIndex, messageIndex, rowMetaJson, stepPluginClasses, xpPluginClasses );
77 |
78 | // Then write to Kafka topic
79 | //
80 | KafkaIO.Write stringsToKafka = KafkaIO.write()
81 | .withBootstrapServers( bootstrapServers )
82 | .withTopic( topic )
83 | .withKeySerializer( StringSerializer.class )
84 | .withValueSerializer( StringSerializer.class );
85 | // TODO: add features like compression
86 | //
87 |
88 | PCollection> kvpCollection = input.apply( ParDo.of( kettleRowToKVStringStringFn ) );
89 | return kvpCollection.apply( stringsToKafka );
90 | } catch ( Exception e ) {
91 | numErrors.inc();
92 | LOG.error( "Error in Beam Kafka output transform", e );
93 | throw new RuntimeException( "Error in Beam Kafka output transform", e );
94 | }
95 | }
96 | }
97 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/core/fn/StringToKettleFn.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.core.fn;
2 |
3 | import org.apache.beam.sdk.metrics.Counter;
4 | import org.apache.beam.sdk.metrics.Metrics;
5 | import org.apache.beam.sdk.transforms.DoFn;
6 | import org.kettle.beam.core.BeamKettle;
7 | import org.kettle.beam.core.KettleRow;
8 | import org.kettle.beam.core.util.JsonRowMeta;
9 | import org.pentaho.di.core.exception.KettleException;
10 | import org.pentaho.di.core.exception.KettleValueException;
11 | import org.pentaho.di.core.row.RowDataUtil;
12 | import org.pentaho.di.core.row.RowMetaInterface;
13 | import org.pentaho.di.core.row.ValueMetaInterface;
14 | import org.pentaho.di.core.row.value.ValueMetaString;
15 | import org.slf4j.Logger;
16 | import org.slf4j.LoggerFactory;
17 |
18 | import java.util.List;
19 |
20 | public class StringToKettleFn extends DoFn {
21 |
22 | private String stepname;
23 | private String rowMetaJson;
24 | private String separator;
25 | private List stepPluginClasses;
26 | private List xpPluginClasses;
27 |
28 | private transient Counter inputCounter;
29 | private transient Counter writtenCounter;
30 |
31 | // Log and count parse errors.
32 | private static final Logger LOG = LoggerFactory.getLogger( StringToKettleFn.class );
33 |
34 | private transient RowMetaInterface rowMeta;
35 |
36 | public StringToKettleFn( String stepname, String rowMetaJson, String separator, List stepPluginClasses, List xpPluginClasses ) {
37 | this.stepname = stepname;
38 | this.rowMetaJson = rowMetaJson;
39 | this.separator = separator;
40 | this.stepPluginClasses = stepPluginClasses;
41 | this.xpPluginClasses = xpPluginClasses;
42 | }
43 |
44 | @Setup
45 | public void setUp() {
46 | try {
47 | inputCounter = Metrics.counter( "input", stepname );
48 | writtenCounter = Metrics.counter( "written", stepname );
49 |
50 | // Initialize Kettle Beam
51 | //
52 | BeamKettle.init( stepPluginClasses, xpPluginClasses );
53 | rowMeta = JsonRowMeta.fromJson( rowMetaJson );
54 |
55 | Metrics.counter( "init", stepname ).inc();
56 | } catch ( Exception e ) {
57 | Metrics.counter( "error", stepname ).inc();
58 | LOG.error( "Error in setup of converting input data into Kettle rows : " + e.getMessage() );
59 | throw new RuntimeException( "Error in setup of converting input data into Kettle rows", e );
60 | }
61 | }
62 |
63 | @ProcessElement
64 | public void processElement( ProcessContext processContext ) {
65 |
66 | try {
67 |
68 | String inputString = processContext.element();
69 | inputCounter.inc();
70 |
71 | String[] components = inputString.split( separator, -1 );
72 |
73 | // TODO: implement enclosure in FileDefinition
74 | //
75 |
76 | Object[] row = RowDataUtil.allocateRowData( rowMeta.size() );
77 | int index = 0;
78 | while ( index < rowMeta.size() && index < components.length ) {
79 | String sourceString = components[ index ];
80 | ValueMetaInterface valueMeta = rowMeta.getValueMeta( index );
81 | ValueMetaInterface stringMeta = new ValueMetaString( "SourceString" );
82 | stringMeta.setConversionMask( valueMeta.getConversionMask() );
83 | try {
84 | row[ index ] = valueMeta.convertDataFromString( sourceString, stringMeta, null, null, ValueMetaInterface.TRIM_TYPE_NONE );
85 | } catch ( KettleValueException ve ) {
86 | throw new KettleException( "Unable to convert value '" + sourceString + "' to value : " + valueMeta.toStringMeta(), ve );
87 | }
88 | index++;
89 | }
90 |
91 | // Pass the row to the process context
92 | //
93 | processContext.output( new KettleRow( row ) );
94 | writtenCounter.inc();
95 |
96 | } catch ( Exception e ) {
97 | Metrics.counter( "error", stepname ).inc();
98 | LOG.error( "Error converting input data into Kettle rows " + processContext.element() + ", " + e.getMessage() );
99 | throw new RuntimeException( "Error converting input data into Kettle rows", e );
100 |
101 | }
102 | }
103 |
104 |
105 | }
106 |
--------------------------------------------------------------------------------
/src/test/java/core/metastore/SerializableMetaStoreTest.java:
--------------------------------------------------------------------------------
1 | package core.metastore;
2 |
3 | import org.junit.Test;
4 | import org.kettle.beam.core.metastore.SerializableMetaStore;
5 | import org.pentaho.metastore.api.IMetaStore;
6 | import org.pentaho.metastore.api.IMetaStoreAttribute;
7 | import org.pentaho.metastore.api.IMetaStoreElement;
8 | import org.pentaho.metastore.api.IMetaStoreElementType;
9 | import org.pentaho.metastore.api.exceptions.MetaStoreException;
10 |
11 | import java.util.List;
12 |
13 | import static org.junit.Assert.assertEquals;
14 |
15 | public class SerializableMetaStoreTest {
16 |
17 | public static final String NAMESPACE = "kettle";
18 | public static final String TYPE_A = "Type A";
19 | public static final String DESCRIPTION_OF_TYPE_A = "Description of Type A";
20 | public static final String ELEMENT_A_1 = "Element A1";
21 | public static final String DESCRIPTION_OF_ELEMENT_A_1 = "Description of Element A1";
22 | public static final long SOME_LONG = 1234L;
23 |
24 | @Test
25 | public void testToJson() throws Exception {
26 |
27 | SerializableMetaStore store1 = new SerializableMetaStore();
28 | store1.setName( "Serialization test metastore" );
29 | store1.createNamespace( NAMESPACE );
30 |
31 | IMetaStoreElementType typeA = store1.newElementType( NAMESPACE );
32 | typeA.setName( TYPE_A );
33 | typeA.setDescription( DESCRIPTION_OF_TYPE_A );
34 | store1.createElementType( NAMESPACE, typeA );
35 |
36 | IMetaStoreElement elementA1 = store1.newElement( typeA, ELEMENT_A_1, null );
37 | elementA1.addChild( store1.newAttribute( "name", ELEMENT_A_1 ) );
38 | elementA1.addChild( store1.newAttribute( "someInt", SOME_LONG ) );
39 | store1.createElement( NAMESPACE, typeA, elementA1 );
40 |
41 | String json = store1.toJson();
42 |
43 | SerializableMetaStore store2 = new SerializableMetaStore( json );
44 | assertEqualMetastores(store1, store2);
45 | }
46 |
47 | private void assertEqualMetastores( IMetaStore store1, IMetaStore store2 ) throws MetaStoreException {
48 | List namespaces1 = store1.getNamespaces();
49 | List namespaces2 = store2.getNamespaces();
50 | assertEquals( namespaces1.size(), namespaces2.size());
51 |
52 | for (int n=0;n elementTypes1 = store1.getElementTypes( namespace1 );
58 | List elementTypes2 = store2.getElementTypes( namespace2 );
59 | assertEquals( elementTypes1.size(), elementTypes2.size() );
60 | for (int t=0;t elements1 = store1.getElements( namespace1, elementType1 );
67 | List elements2 = store2.getElements( namespace2, elementType2 );
68 | assertEquals( elements1.size(), elements2.size() );
69 | for (int e=0;e children1 = element1.getChildren();
87 | List children2 = element2.getChildren();
88 | for (int c=0;c databases, IMetaStore metaStore ) throws KettleXMLException {
81 | fieldName = XMLHandler.getTagValue( stepnode, FIELD_NAME );
82 | readingTimestamp = "Y".equalsIgnoreCase( XMLHandler.getTagValue( stepnode, READ_TIMESTAMP ) );
83 | }
84 |
85 |
86 | /**
87 | * Gets fieldName
88 | *
89 | * @return value of fieldName
90 | */
91 | public String getFieldName() {
92 | return fieldName;
93 | }
94 |
95 | /**
96 | * @param fieldName The fieldName to set
97 | */
98 | public void setFieldName( String fieldName ) {
99 | this.fieldName = fieldName;
100 | }
101 |
102 | /**
103 | * Gets readingTimestamp
104 | *
105 | * @return value of readingTimestamp
106 | */
107 | public boolean isReadingTimestamp() {
108 | return readingTimestamp;
109 | }
110 |
111 | /**
112 | * @param readingTimestamp The readingTimestamp to set
113 | */
114 | public void setReadingTimestamp( boolean readingTimestamp ) {
115 | this.readingTimestamp = readingTimestamp;
116 | }
117 | }
118 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/steps/pubsub/BeamPublishMeta.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.steps.pubsub;
2 |
3 | import org.apache.commons.lang.StringUtils;
4 | import org.kettle.beam.core.BeamDefaults;
5 | import org.pentaho.di.core.annotations.Step;
6 | import org.pentaho.di.core.database.DatabaseMeta;
7 | import org.pentaho.di.core.exception.KettleException;
8 | import org.pentaho.di.core.exception.KettleStepException;
9 | import org.pentaho.di.core.exception.KettleXMLException;
10 | import org.pentaho.di.core.row.RowMetaInterface;
11 | import org.pentaho.di.core.row.ValueMetaInterface;
12 | import org.pentaho.di.core.row.value.ValueMetaSerializable;
13 | import org.pentaho.di.core.row.value.ValueMetaString;
14 | import org.pentaho.di.core.variables.VariableSpace;
15 | import org.pentaho.di.core.xml.XMLHandler;
16 | import org.pentaho.di.repository.Repository;
17 | import org.pentaho.di.trans.Trans;
18 | import org.pentaho.di.trans.TransMeta;
19 | import org.pentaho.di.trans.step.BaseStepMeta;
20 | import org.pentaho.di.trans.step.StepDataInterface;
21 | import org.pentaho.di.trans.step.StepInterface;
22 | import org.pentaho.di.trans.step.StepMeta;
23 | import org.pentaho.di.trans.step.StepMetaInterface;
24 | import org.pentaho.metastore.api.IMetaStore;
25 | import org.w3c.dom.Node;
26 |
27 | import java.util.List;
28 |
29 | @Step(
30 | id = "BeamPublish",
31 | name = "Beam GCP Pub/Sub : Publish",
32 | description = "Publish to a Pub/Sub topic",
33 | image = "beam-gcp-pubsub-publish.svg",
34 | categoryDescription = "Big Data"
35 | )
36 | public class BeamPublishMeta extends BaseStepMeta implements StepMetaInterface {
37 |
38 | public static final String TOPIC = "topic";
39 | public static final String MESSAGE_TYPE = "message_type";
40 | public static final String MESSAGE_FIELD = "message_field";
41 |
42 | private String topic;
43 | private String messageType;
44 | private String messageField;
45 |
46 | public BeamPublishMeta() {
47 | super();
48 | }
49 |
50 | @Override public void setDefault() {
51 | topic = "Topic";
52 | messageType = "String";
53 | messageField = "message";
54 | }
55 |
56 | @Override public StepInterface getStep( StepMeta stepMeta, StepDataInterface stepDataInterface, int copyNr, TransMeta transMeta, Trans trans ) {
57 | return new BeamSubscribe( stepMeta, stepDataInterface, copyNr, transMeta, trans );
58 | }
59 |
60 | @Override public StepDataInterface getStepData() {
61 | return new BeamPublishData();
62 | }
63 |
64 | @Override public String getDialogClassName() {
65 | return BeamPublishDialog.class.getName();
66 | }
67 |
68 | @Override public void getFields( RowMetaInterface inputRowMeta, String name, RowMetaInterface[] info, StepMeta nextStep, VariableSpace space, Repository repository, IMetaStore metaStore )
69 | throws KettleStepException {
70 |
71 | // No output
72 | //
73 | inputRowMeta.clear();
74 | }
75 |
76 | @Override public String getXML() throws KettleException {
77 | StringBuffer xml = new StringBuffer();
78 | xml.append( XMLHandler.addTagValue( TOPIC, topic ) );
79 | xml.append( XMLHandler.addTagValue( MESSAGE_TYPE, messageType ) );
80 | xml.append( XMLHandler.addTagValue( MESSAGE_FIELD, messageField ) );
81 | return xml.toString();
82 | }
83 |
84 | @Override public void loadXML( Node stepnode, List databases, IMetaStore metaStore ) throws KettleXMLException {
85 | topic = XMLHandler.getTagValue( stepnode, TOPIC );
86 | messageType = XMLHandler.getTagValue( stepnode, MESSAGE_TYPE );
87 | messageField = XMLHandler.getTagValue( stepnode, MESSAGE_FIELD );
88 | }
89 |
90 |
91 | /**
92 | * Gets topic
93 | *
94 | * @return value of topic
95 | */
96 | public String getTopic() {
97 | return topic;
98 | }
99 |
100 | /**
101 | * @param topic The topic to set
102 | */
103 | public void setTopic( String topic ) {
104 | this.topic = topic;
105 | }
106 |
107 | /**
108 | * Gets messageType
109 | *
110 | * @return value of messageType
111 | */
112 | public String getMessageType() {
113 | return messageType;
114 | }
115 |
116 | /**
117 | * @param messageType The messageType to set
118 | */
119 | public void setMessageType( String messageType ) {
120 | this.messageType = messageType;
121 | }
122 |
123 | /**
124 | * Gets messageField
125 | *
126 | * @return value of messageField
127 | */
128 | public String getMessageField() {
129 | return messageField;
130 | }
131 |
132 | /**
133 | * @param messageField The messageField to set
134 | */
135 | public void setMessageField( String messageField ) {
136 | this.messageField = messageField;
137 | }
138 | }
139 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/pipeline/handler/BeamOutputStepHandler.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.pipeline.handler;
2 |
3 | import org.apache.beam.sdk.Pipeline;
4 | import org.apache.beam.sdk.values.PCollection;
5 | import org.apache.commons.lang.StringUtils;
6 | import org.kettle.beam.core.KettleRow;
7 | import org.kettle.beam.core.transform.BeamOutputTransform;
8 | import org.kettle.beam.core.util.JsonRowMeta;
9 | import org.kettle.beam.metastore.BeamJobConfig;
10 | import org.kettle.beam.metastore.FieldDefinition;
11 | import org.kettle.beam.metastore.FileDefinition;
12 | import org.kettle.beam.steps.io.BeamOutputMeta;
13 | import org.pentaho.di.core.exception.KettleException;
14 | import org.pentaho.di.core.exception.KettleStepException;
15 | import org.pentaho.di.core.logging.LogChannelInterface;
16 | import org.pentaho.di.core.row.RowMetaInterface;
17 | import org.pentaho.di.core.row.ValueMetaInterface;
18 | import org.pentaho.di.trans.TransMeta;
19 | import org.pentaho.di.trans.step.StepMeta;
20 | import org.pentaho.metastore.api.IMetaStore;
21 |
22 | import java.util.List;
23 | import java.util.Map;
24 |
25 | public class BeamOutputStepHandler extends BeamBaseStepHandler implements BeamStepHandler {
26 |
27 | public BeamOutputStepHandler( BeamJobConfig beamJobConfig, IMetaStore metaStore, TransMeta transMeta, List stepPluginClasses, List xpPluginClasses ) {
28 | super( beamJobConfig, false, true, metaStore, transMeta, stepPluginClasses, xpPluginClasses );
29 | }
30 |
31 | @Override public void handleStep( LogChannelInterface log, StepMeta beamOutputStepMeta, Map> stepCollectionMap,
32 | Pipeline pipeline, RowMetaInterface rowMeta, List previousSteps,
33 | PCollection input ) throws KettleException {
34 |
35 | BeamOutputMeta beamOutputMeta = (BeamOutputMeta) beamOutputStepMeta.getStepMetaInterface();
36 | FileDefinition outputFileDefinition;
37 | if ( StringUtils.isEmpty( beamOutputMeta.getFileDescriptionName() ) ) {
38 | // Create a default file definition using standard output and sane defaults...
39 | //
40 | outputFileDefinition = getDefaultFileDefition( beamOutputStepMeta );
41 | } else {
42 | outputFileDefinition = beamOutputMeta.loadFileDefinition( metaStore );
43 | }
44 |
45 | // Empty file definition? Add all fields in the output
46 | //
47 | addAllFieldsToEmptyFileDefinition( rowMeta, outputFileDefinition );
48 |
49 | // Apply the output transform from KettleRow to PDone
50 | //
51 | if ( outputFileDefinition == null ) {
52 | throw new KettleException( "We couldn't find or load the Beam Output step file definition" );
53 | }
54 | if ( rowMeta == null || rowMeta.isEmpty() ) {
55 | throw new KettleException( "No output fields found in the file definition or from previous steps" );
56 | }
57 |
58 | BeamOutputTransform beamOutputTransform = new BeamOutputTransform(
59 | beamOutputStepMeta.getName(),
60 | transMeta.environmentSubstitute( beamOutputMeta.getOutputLocation() ),
61 | transMeta.environmentSubstitute( beamOutputMeta.getFilePrefix() ),
62 | transMeta.environmentSubstitute( beamOutputMeta.getFileSuffix() ),
63 | transMeta.environmentSubstitute( outputFileDefinition.getSeparator() ),
64 | transMeta.environmentSubstitute( outputFileDefinition.getEnclosure() ),
65 | beamOutputMeta.isWindowed(),
66 | JsonRowMeta.toJson( rowMeta ),
67 | stepPluginClasses,
68 | xpPluginClasses
69 | );
70 |
71 | // Which step do we apply this transform to?
72 | // Ignore info hops until we figure that out.
73 | //
74 | if ( previousSteps.size() > 1 ) {
75 | throw new KettleException( "Combining data from multiple steps is not supported yet!" );
76 | }
77 | StepMeta previousStep = previousSteps.get( 0 );
78 |
79 | // No need to store this, it's PDone.
80 | //
81 | input.apply( beamOutputTransform );
82 | log.logBasic( "Handled step (OUTPUT) : " + beamOutputStepMeta.getName() + ", gets data from " + previousStep.getName() );
83 | }
84 |
85 | private FileDefinition getDefaultFileDefition( StepMeta beamOutputStepMeta ) throws KettleStepException {
86 | FileDefinition fileDefinition = new FileDefinition();
87 |
88 | fileDefinition.setName( "Default" );
89 | fileDefinition.setEnclosure( "\"" );
90 | fileDefinition.setSeparator( "," );
91 |
92 | return fileDefinition;
93 | }
94 |
95 | private void addAllFieldsToEmptyFileDefinition( RowMetaInterface rowMeta, FileDefinition fileDefinition ) throws KettleStepException {
96 | if ( fileDefinition.getFieldDefinitions().isEmpty() ) {
97 | for ( ValueMetaInterface valueMeta : rowMeta.getValueMetaList() ) {
98 | fileDefinition.getFieldDefinitions().add( new FieldDefinition(
99 | valueMeta.getName(),
100 | valueMeta.getTypeDesc(),
101 | valueMeta.getLength(),
102 | valueMeta.getPrecision(),
103 | valueMeta.getConversionMask()
104 | )
105 | );
106 | }
107 | }
108 | }
109 |
110 | }
111 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/core/fn/WindowInfoFn.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.core.fn;
2 |
3 | import org.apache.beam.sdk.metrics.Counter;
4 | import org.apache.beam.sdk.metrics.Metrics;
5 | import org.apache.beam.sdk.transforms.DoFn;
6 | import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
7 | import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
8 | import org.apache.commons.lang.StringUtils;
9 | import org.joda.time.Instant;
10 | import org.kettle.beam.core.BeamKettle;
11 | import org.kettle.beam.core.KettleRow;
12 | import org.kettle.beam.core.util.JsonRowMeta;
13 | import org.pentaho.di.core.row.RowDataUtil;
14 | import org.pentaho.di.core.row.RowMetaInterface;
15 | import org.pentaho.di.core.row.ValueMetaInterface;
16 | import org.slf4j.Logger;
17 | import org.slf4j.LoggerFactory;
18 |
19 | import java.util.List;
20 |
21 | public class WindowInfoFn extends DoFn {
22 |
23 | private String stepname;
24 | private String maxWindowField;
25 | private String startWindowField;
26 | private String endWindowField;
27 | private String rowMetaJson;
28 | private List stepPluginClasses;
29 | private List xpPluginClasses;
30 |
31 | private transient Counter initCounter;
32 | private transient Counter readCounter;
33 | private transient Counter writtenCounter;
34 | private transient Counter errorCounter;
35 |
36 | private transient int fieldIndex;
37 |
38 | // Log and count parse errors.
39 | private static final Logger LOG = LoggerFactory.getLogger( WindowInfoFn.class );
40 |
41 | private transient RowMetaInterface inputRowMeta;
42 | private transient ValueMetaInterface fieldValueMeta;
43 |
44 | public WindowInfoFn( String stepname, String maxWindowField, String startWindowField, String endWindowField, String rowMetaJson, List stepPluginClasses, List xpPluginClasses ) {
45 | this.stepname = stepname;
46 | this.maxWindowField = maxWindowField;
47 | this.startWindowField = startWindowField;
48 | this.endWindowField = endWindowField;
49 | this.rowMetaJson = rowMetaJson;
50 | this.stepPluginClasses = stepPluginClasses;
51 | this.xpPluginClasses = xpPluginClasses;
52 | }
53 |
54 | @Setup
55 | public void setUp() {
56 | try {
57 | readCounter = Metrics.counter( "read", stepname );
58 | writtenCounter = Metrics.counter( "written", stepname );
59 | errorCounter = Metrics.counter( "error", stepname );
60 |
61 | // Initialize Kettle Beam
62 | //
63 | BeamKettle.init( stepPluginClasses, xpPluginClasses );
64 | inputRowMeta = JsonRowMeta.fromJson( rowMetaJson );
65 |
66 | Metrics.counter( "init", stepname ).inc();
67 | } catch(Exception e) {
68 | errorCounter.inc();
69 | LOG.error( "Error in setup of adding window information to rows : " + e.getMessage() );
70 | throw new RuntimeException( "Error in setup of adding window information to rows", e );
71 | }
72 | }
73 |
74 |
75 | @ProcessElement
76 | public void processElement( ProcessContext processContext, BoundedWindow window ) {
77 |
78 | try {
79 |
80 | KettleRow kettleRow = processContext.element();
81 | readCounter.inc();
82 |
83 | Instant instant = window.maxTimestamp();
84 |
85 | Object[] outputRow = RowDataUtil.createResizedCopy( kettleRow.getRow(), inputRowMeta.size()+3 );
86 |
87 | int fieldIndex = inputRowMeta.size();
88 |
89 | // Kettle "Date" type field output: java.util.Date.
90 | // Use the last field in the output
91 | //
92 | if ( StringUtils.isNotEmpty( startWindowField ) ) {
93 | if ( window instanceof IntervalWindow ) {
94 | IntervalWindow intervalWindow = (IntervalWindow) window;
95 | Instant start = intervalWindow.start();
96 | if ( start != null ) {
97 | outputRow[ fieldIndex ] = start.toDate();
98 | }
99 | }
100 | fieldIndex++;
101 | }
102 | if ( StringUtils.isNotEmpty( endWindowField ) ) {
103 | if ( window instanceof IntervalWindow ) {
104 | IntervalWindow intervalWindow = (IntervalWindow) window;
105 | Instant end = intervalWindow.end();
106 | if ( end != null ) {
107 | outputRow[ fieldIndex ] = end.toDate();
108 | }
109 | }
110 | fieldIndex++;
111 | }
112 |
113 | if ( StringUtils.isNotEmpty( maxWindowField ) ) {
114 | Instant maxTimestamp = window.maxTimestamp();
115 | if ( maxTimestamp != null ) {
116 | outputRow[ fieldIndex ] = maxTimestamp.toDate();
117 | }
118 | fieldIndex++;
119 | }
120 |
121 | // Pass the new row to the process context
122 | //
123 | KettleRow outputKettleRow = new KettleRow( outputRow );
124 | processContext.outputWithTimestamp( outputKettleRow, instant );
125 | writtenCounter.inc();
126 |
127 | } catch ( Exception e ) {
128 | errorCounter.inc();
129 | LOG.error( "Error adding window information to rows : " + processContext.element() + ", " + e.getMessage() );
130 | throw new RuntimeException( "Error adding window information to rows", e );
131 | }
132 | }
133 |
134 | }
135 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/steps/kafka/BeamProduceMeta.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.steps.kafka;
2 |
3 | import org.pentaho.di.core.annotations.Step;
4 | import org.pentaho.di.core.database.DatabaseMeta;
5 | import org.pentaho.di.core.exception.KettleException;
6 | import org.pentaho.di.core.exception.KettleStepException;
7 | import org.pentaho.di.core.exception.KettleXMLException;
8 | import org.pentaho.di.core.row.RowMetaInterface;
9 | import org.pentaho.di.core.variables.VariableSpace;
10 | import org.pentaho.di.core.xml.XMLHandler;
11 | import org.pentaho.di.repository.Repository;
12 | import org.pentaho.di.trans.Trans;
13 | import org.pentaho.di.trans.TransMeta;
14 | import org.pentaho.di.trans.step.BaseStepMeta;
15 | import org.pentaho.di.trans.step.StepDataInterface;
16 | import org.pentaho.di.trans.step.StepInterface;
17 | import org.pentaho.di.trans.step.StepMeta;
18 | import org.pentaho.di.trans.step.StepMetaInterface;
19 | import org.pentaho.di.trans.steps.dummytrans.DummyTransData;
20 | import org.pentaho.metastore.api.IMetaStore;
21 | import org.w3c.dom.Node;
22 |
23 | import java.util.List;
24 |
25 | @Step(
26 | id = "BeamKafkaProduce",
27 | name = "Beam Kafka Produce",
28 | description = "Send messages to a Kafka Topic (Producer)",
29 | image = "beam-kafka-output.svg",
30 | categoryDescription = "Big Data"
31 | )
32 | public class BeamProduceMeta extends BaseStepMeta implements StepMetaInterface {
33 |
34 | public static final String BOOTSTRAP_SERVERS = "bootstrap_servers";
35 | public static final String TOPIC = "topic";
36 | public static final String KEY_FIELD = "key_field";
37 | public static final String MESSAGE_FIELD = "message_field";
38 |
39 | private String bootstrapServers;
40 | private String topic;
41 | private String keyField;
42 | private String messageField;
43 |
44 | public BeamProduceMeta() {
45 | super();
46 | }
47 |
48 | @Override public void setDefault() {
49 | bootstrapServers = "bootstrapServer1:9001,bootstrapServer2:9001";
50 | topic = "Topic1";
51 | keyField = "";
52 | messageField = "";
53 | }
54 |
55 | @Override public StepInterface getStep( StepMeta stepMeta, StepDataInterface stepDataInterface, int copyNr, TransMeta transMeta, Trans trans ) {
56 | return new BeamConsume( stepMeta, stepDataInterface, copyNr, transMeta, trans );
57 | }
58 |
59 | @Override public StepDataInterface getStepData() {
60 | return new DummyTransData();
61 | }
62 |
63 | @Override public String getDialogClassName() {
64 | return BeamProduceDialog.class.getName();
65 | }
66 |
67 | @Override public void getFields( RowMetaInterface inputRowMeta, String name, RowMetaInterface[] info, StepMeta nextStep, VariableSpace space, Repository repository, IMetaStore metaStore )
68 | throws KettleStepException {
69 |
70 | // No output
71 | //
72 | inputRowMeta.clear();
73 | }
74 |
75 | @Override public String getXML() throws KettleException {
76 | StringBuffer xml = new StringBuffer();
77 | xml.append( XMLHandler.addTagValue( BOOTSTRAP_SERVERS, bootstrapServers ) );
78 | xml.append( XMLHandler.addTagValue( TOPIC, topic ) );
79 | xml.append( XMLHandler.addTagValue( KEY_FIELD, keyField ) );
80 | xml.append( XMLHandler.addTagValue( MESSAGE_FIELD, messageField ) );
81 | return xml.toString();
82 | }
83 |
84 | @Override public void loadXML( Node stepnode, List databases, IMetaStore metaStore ) throws KettleXMLException {
85 | bootstrapServers = XMLHandler.getTagValue( stepnode, BOOTSTRAP_SERVERS );
86 | topic = XMLHandler.getTagValue( stepnode, TOPIC );
87 | keyField = XMLHandler.getTagValue( stepnode, KEY_FIELD );
88 | messageField = XMLHandler.getTagValue( stepnode, MESSAGE_FIELD );
89 | }
90 |
91 |
92 | /**
93 | * Gets topic
94 | *
95 | * @return value of topic
96 | */
97 | public String getTopic() {
98 | return topic;
99 | }
100 |
101 | /**
102 | * @param topic The topic to set
103 | */
104 | public void setTopic( String topic ) {
105 | this.topic = topic;
106 | }
107 |
108 | /**
109 | * Gets keyField
110 | *
111 | * @return value of keyField
112 | */
113 | public String getKeyField() {
114 | return keyField;
115 | }
116 |
117 | /**
118 | * @param keyField The keyField to set
119 | */
120 | public void setKeyField( String keyField ) {
121 | this.keyField = keyField;
122 | }
123 |
124 | /**
125 | * Gets messageField
126 | *
127 | * @return value of messageField
128 | */
129 | public String getMessageField() {
130 | return messageField;
131 | }
132 |
133 | /**
134 | * @param messageField The messageField to set
135 | */
136 | public void setMessageField( String messageField ) {
137 | this.messageField = messageField;
138 | }
139 |
140 | /**
141 | * Gets bootstrapServers
142 | *
143 | * @return value of bootstrapServers
144 | */
145 | public String getBootstrapServers() {
146 | return bootstrapServers;
147 | }
148 |
149 | /**
150 | * @param bootstrapServers The bootstrapServers to set
151 | */
152 | public void setBootstrapServers( String bootstrapServers ) {
153 | this.bootstrapServers = bootstrapServers;
154 | }
155 | }
156 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/core/fn/AssemblerFn.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.core.fn;
2 |
3 | import org.apache.beam.sdk.metrics.Counter;
4 | import org.apache.beam.sdk.metrics.Metrics;
5 | import org.apache.beam.sdk.transforms.DoFn;
6 | import org.apache.beam.sdk.values.KV;
7 | import org.kettle.beam.core.BeamKettle;
8 | import org.kettle.beam.core.KettleRow;
9 | import org.kettle.beam.core.util.JsonRowMeta;
10 | import org.pentaho.di.core.row.RowDataUtil;
11 | import org.pentaho.di.core.row.RowMetaInterface;
12 | import org.slf4j.Logger;
13 | import org.slf4j.LoggerFactory;
14 |
15 | import java.util.List;
16 |
17 | public class AssemblerFn extends DoFn>, KettleRow> {
18 |
19 | private String outputRowMetaJson;
20 | private String leftKRowMetaJson;
21 | private String leftVRowMetaJson;
22 | private String rightVRowMetaJson;
23 | private String counterName;
24 | private List stepPluginClasses;
25 | private ListxpPluginClasses;
26 |
27 | private static final Logger LOG = LoggerFactory.getLogger( AssemblerFn.class );
28 |
29 | private transient RowMetaInterface outputRowMeta;
30 | private transient RowMetaInterface leftKRowMeta;
31 | private transient RowMetaInterface leftVRowMeta;
32 | private transient RowMetaInterface rightVRowMeta;
33 |
34 | private transient Counter initCounter;
35 | private transient Counter writtenCounter;
36 | private transient Counter errorCounter;
37 |
38 | public AssemblerFn() {
39 | }
40 |
41 | public AssemblerFn( String outputRowMetaJson, String leftKRowMetaJson, String leftVRowMetaJson, String rightVRowMetaJson, String counterName,
42 | List stepPluginClasses, ListxpPluginClasses) {
43 | this.outputRowMetaJson = outputRowMetaJson;
44 | this.leftKRowMetaJson = leftKRowMetaJson;
45 | this.leftVRowMetaJson = leftVRowMetaJson;
46 | this.rightVRowMetaJson = rightVRowMetaJson;
47 | this.counterName = counterName;
48 | this.stepPluginClasses = stepPluginClasses;
49 | this.xpPluginClasses = xpPluginClasses;
50 | }
51 |
52 | @Setup
53 | public void setUp() {
54 | try {
55 | writtenCounter = Metrics.counter( "written", counterName );
56 | errorCounter = Metrics.counter( "error", counterName );
57 |
58 | // Initialize Kettle Beam
59 | //
60 | BeamKettle.init( stepPluginClasses, xpPluginClasses );
61 | outputRowMeta = JsonRowMeta.fromJson( outputRowMetaJson );
62 | leftKRowMeta = JsonRowMeta.fromJson( leftKRowMetaJson );
63 | leftVRowMeta = JsonRowMeta.fromJson( leftVRowMetaJson );
64 | rightVRowMeta = JsonRowMeta.fromJson( rightVRowMetaJson );
65 |
66 | Metrics.counter( "init", counterName ).inc();
67 | } catch(Exception e) {
68 | errorCounter.inc();
69 | LOG.error( "Error initializing assembling rows", e);
70 | throw new RuntimeException( "Error initializing assembling output KV>", e );
71 | }
72 | }
73 |
74 | @ProcessElement
75 | public void processElement( ProcessContext processContext ) {
76 |
77 | try {
78 |
79 | KV> element = processContext.element();
80 | KV value = element.getValue();
81 |
82 | KettleRow key = element.getKey();
83 | KettleRow leftValue = value.getKey();
84 | KettleRow rightValue = value.getValue();
85 |
86 | Object[] outputRow = RowDataUtil.allocateRowData( outputRowMeta.size() );
87 | int index = 0;
88 |
89 | // Kettle style, first the left values
90 | //
91 | if (leftValue.allNull()) {
92 | index+=leftVRowMeta.size();
93 | } else {
94 | for ( int i = 0; i < leftVRowMeta.size(); i++ ) {
95 | outputRow[ index++ ] = leftValue.getRow()[ i ];
96 | }
97 | }
98 |
99 | // Now the left key
100 | //
101 | if (leftValue.allNull()) {
102 | index+=leftKRowMeta.size();
103 | } else {
104 | for ( int i = 0; i < leftKRowMeta.size(); i++ ) {
105 | outputRow[ index++ ] = key.getRow()[ i ];
106 | }
107 | }
108 |
109 | // Then the right key
110 | //
111 | if (rightValue.allNull()) {
112 | // No right key given if the value is null
113 | //
114 | index+=leftKRowMeta.size();
115 | } else {
116 | for ( int i = 0; i < leftKRowMeta.size(); i++ ) {
117 | outputRow[ index++ ] = key.getRow()[ i ];
118 | }
119 | }
120 |
121 | // Finally the right values
122 | //
123 | if (rightValue.allNull()) {
124 | index+=rightVRowMeta.size();
125 | } else {
126 | for ( int i = 0; i < rightVRowMeta.size(); i++ ) {
127 | outputRow[ index++ ] = rightValue.getRow()[ i ];
128 | }
129 | }
130 |
131 | // System.out.println("Assembled row : "+outputRowMeta.getString(outputRow));
132 |
133 | processContext.output( new KettleRow( outputRow ) );
134 | writtenCounter.inc();
135 |
136 | } catch(Exception e) {
137 | errorCounter.inc();
138 | LOG.error( "Error assembling rows", e);
139 | throw new RuntimeException( "Error assembling output KV>", e );
140 | }
141 | }
142 | }
143 |
144 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/steps/io/BeamInputMeta.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.steps.io;
2 |
3 | import org.apache.commons.lang.StringUtils;
4 | import org.kettle.beam.metastore.FileDefinition;
5 | import org.pentaho.di.core.annotations.Step;
6 | import org.pentaho.di.core.database.DatabaseMeta;
7 | import org.pentaho.di.core.exception.KettleException;
8 | import org.pentaho.di.core.exception.KettlePluginException;
9 | import org.pentaho.di.core.exception.KettleStepException;
10 | import org.pentaho.di.core.exception.KettleXMLException;
11 | import org.pentaho.di.core.row.RowMetaInterface;
12 | import org.pentaho.di.core.variables.VariableSpace;
13 | import org.pentaho.di.core.xml.XMLHandler;
14 | import org.pentaho.di.repository.Repository;
15 | import org.pentaho.di.trans.Trans;
16 | import org.pentaho.di.trans.TransMeta;
17 | import org.pentaho.di.trans.step.BaseStepMeta;
18 | import org.pentaho.di.trans.step.StepDataInterface;
19 | import org.pentaho.di.trans.step.StepInterface;
20 | import org.pentaho.di.trans.step.StepMeta;
21 | import org.pentaho.di.trans.step.StepMetaInterface;
22 | import org.pentaho.metastore.api.IMetaStore;
23 | import org.pentaho.metastore.persist.MetaStoreFactory;
24 | import org.pentaho.metastore.util.PentahoDefaults;
25 | import org.w3c.dom.Node;
26 |
27 | import java.util.List;
28 |
29 | @Step(
30 | id = "BeamInput",
31 | name = "Beam Input",
32 | description = "Describes a Beam Input",
33 | image = "beam-input.svg",
34 | categoryDescription = "Big Data"
35 | )
36 | public class BeamInputMeta extends BaseStepMeta implements StepMetaInterface {
37 |
38 | public static final String INPUT_LOCATION = "input_location";
39 | public static final String FILE_DESCRIPTION_NAME = "file_description_name";
40 |
41 | private String inputLocation;
42 |
43 | private String fileDescriptionName;
44 |
45 | public BeamInputMeta() {
46 | super();
47 | }
48 |
49 | @Override public void setDefault() {
50 | }
51 |
52 | @Override public StepInterface getStep( StepMeta stepMeta, StepDataInterface stepDataInterface, int copyNr, TransMeta transMeta, Trans trans ) {
53 | return new BeamInput( stepMeta, stepDataInterface, copyNr, transMeta, trans);
54 | }
55 |
56 | @Override public StepDataInterface getStepData() {
57 | return new BeamInputData();
58 | }
59 |
60 | @Override public String getDialogClassName() {
61 | return BeamInputDialog.class.getName();
62 | }
63 |
64 | @Override public void getFields( RowMetaInterface inputRowMeta, String name, RowMetaInterface[] info, StepMeta nextStep, VariableSpace space, Repository repository, IMetaStore metaStore )
65 | throws KettleStepException {
66 |
67 | if (metaStore!=null) {
68 | FileDefinition fileDefinition = loadFileDefinition( metaStore );
69 |
70 | try {
71 | inputRowMeta.clear();
72 | inputRowMeta.addRowMeta( fileDefinition.getRowMeta() );
73 | } catch ( KettlePluginException e ) {
74 | throw new KettleStepException( "Unable to get row layout of file definition '" + fileDefinition.getName() + "'", e );
75 | }
76 | }
77 | }
78 |
79 | public FileDefinition loadFileDefinition(IMetaStore metaStore) throws KettleStepException {
80 | if (StringUtils.isEmpty(fileDescriptionName)) {
81 | throw new KettleStepException("No file description name provided");
82 | }
83 | FileDefinition fileDefinition;
84 | try {
85 | MetaStoreFactory factory = new MetaStoreFactory<>( FileDefinition.class, metaStore, PentahoDefaults.NAMESPACE );
86 | fileDefinition = factory.loadElement( fileDescriptionName );
87 | } catch(Exception e) {
88 | throw new KettleStepException( "Unable to load file description '"+fileDescriptionName+"' from the metastore", e );
89 | }
90 |
91 | return fileDefinition;
92 | }
93 |
94 | @Override public String getXML() throws KettleException {
95 | StringBuffer xml = new StringBuffer( );
96 |
97 | xml.append( XMLHandler.addTagValue( INPUT_LOCATION, inputLocation ) );
98 | xml.append( XMLHandler.addTagValue( FILE_DESCRIPTION_NAME, fileDescriptionName) );
99 |
100 | return xml.toString();
101 | }
102 |
103 | @Override public void loadXML( Node stepnode, List databases, IMetaStore metaStore ) throws KettleXMLException {
104 |
105 | inputLocation = XMLHandler.getTagValue( stepnode, INPUT_LOCATION );
106 | fileDescriptionName = XMLHandler.getTagValue( stepnode, FILE_DESCRIPTION_NAME );
107 |
108 | }
109 |
110 |
111 | /**
112 | * Gets inputLocation
113 | *
114 | * @return value of inputLocation
115 | */
116 | public String getInputLocation() {
117 | return inputLocation;
118 | }
119 |
120 | /**
121 | * @param inputLocation The inputLocation to set
122 | */
123 | public void setInputLocation( String inputLocation ) {
124 | this.inputLocation = inputLocation;
125 | }
126 |
127 | /**
128 | * Gets fileDescriptionName
129 | *
130 | * @return value of fileDescriptionName
131 | */
132 | public String getFileDescriptionName() {
133 | return fileDescriptionName;
134 | }
135 |
136 | /**
137 | * @param fileDescriptionName The fileDescriptionName to set
138 | */
139 | public void setFileDescriptionName( String fileDescriptionName ) {
140 | this.fileDescriptionName = fileDescriptionName;
141 | }
142 |
143 | }
144 |
--------------------------------------------------------------------------------
/src/main/java/org/kettle/beam/core/fn/TimestampFn.java:
--------------------------------------------------------------------------------
1 | package org.kettle.beam.core.fn;
2 |
3 | import org.apache.beam.sdk.metrics.Counter;
4 | import org.apache.beam.sdk.metrics.Metrics;
5 | import org.apache.beam.sdk.transforms.DoFn;
6 | import org.apache.commons.lang.StringUtils;
7 | import org.joda.time.Duration;
8 | import org.joda.time.Instant;
9 | import org.kettle.beam.core.BeamKettle;
10 | import org.kettle.beam.core.KettleRow;
11 | import org.kettle.beam.core.util.JsonRowMeta;
12 | import org.pentaho.di.core.exception.KettleException;
13 | import org.pentaho.di.core.row.RowDataUtil;
14 | import org.pentaho.di.core.row.RowMetaInterface;
15 | import org.pentaho.di.core.row.ValueMetaInterface;
16 | import org.pentaho.di.core.row.value.ValueMetaTimestamp;
17 | import org.slf4j.Logger;
18 | import org.slf4j.LoggerFactory;
19 |
20 | import java.util.Date;
21 | import java.util.List;
22 |
23 | public class TimestampFn extends DoFn {
24 |
25 | private String stepname;
26 | private String rowMetaJson;
27 | private String fieldName;
28 | private final boolean getTimestamp;
29 | private List stepPluginClasses;
30 | private List xpPluginClasses;
31 |
32 | private transient Counter readCounter;
33 | private transient Counter writtenCounter;
34 | private transient Counter errorCounter;
35 |
36 | private transient int fieldIndex;
37 |
38 | // Log and count parse errors.
39 | private static final Logger LOG = LoggerFactory.getLogger( TimestampFn.class );
40 |
41 | private transient RowMetaInterface inputRowMeta;
42 | private transient ValueMetaInterface fieldValueMeta;
43 |
44 | public TimestampFn( String stepname, String rowMetaJson, String fieldName, boolean getTimestamp, List stepPluginClasses, List xpPluginClasses ) {
45 | this.stepname = stepname;
46 | this.rowMetaJson = rowMetaJson;
47 | this.fieldName = fieldName;
48 | this.getTimestamp = getTimestamp;
49 | this.stepPluginClasses = stepPluginClasses;
50 | this.xpPluginClasses = xpPluginClasses;
51 | }
52 |
53 | @Setup
54 | public void setUp() {
55 | try {
56 | // Initialize Kettle Beam
57 | //
58 | BeamKettle.init( stepPluginClasses, xpPluginClasses );
59 |
60 | inputRowMeta = JsonRowMeta.fromJson( rowMetaJson );
61 |
62 | readCounter = Metrics.counter( "read", stepname );
63 | writtenCounter = Metrics.counter( "written", stepname );
64 | errorCounter = Metrics.counter( "error", stepname );
65 |
66 | fieldIndex = -1;
67 | if ( !getTimestamp && StringUtils.isNotEmpty( fieldName ) ) {
68 | fieldIndex = inputRowMeta.indexOfValue( fieldName );
69 | if ( fieldIndex < 0 ) {
70 | throw new RuntimeException( "Field '" + fieldName + "' couldn't be found in put : " + inputRowMeta.toString() );
71 | }
72 | fieldValueMeta = inputRowMeta.getValueMeta( fieldIndex );
73 | }
74 |
75 | Metrics.counter( "init", stepname ).inc();
76 | } catch(Exception e) {
77 | errorCounter.inc();
78 | LOG.error( "Error in setup of adding timestamp to rows : " + e.getMessage() );
79 | throw new RuntimeException( "Error setup of adding timestamp to rows", e );
80 | }
81 | }
82 |
83 |
84 | @ProcessElement
85 | public void processElement( ProcessContext processContext ) {
86 |
87 | try {
88 |
89 | KettleRow kettleRow = processContext.element();
90 | readCounter.inc();
91 |
92 | // The instant
93 | //
94 | Instant instant;
95 |
96 | if ( getTimestamp ) {
97 | instant = processContext.timestamp();
98 |
99 | // Add one row to the stream.
100 | //
101 | Object[] outputRow = RowDataUtil.createResizedCopy( kettleRow.getRow(), inputRowMeta.size() + 1 );
102 |
103 | // Kettle "Date" type field output: java.util.Date.
104 | // Use the last field in the output
105 | //
106 | outputRow[ inputRowMeta.size() ] = instant.toDate();
107 | kettleRow = new KettleRow( outputRow );
108 | } else {
109 | if ( fieldIndex < 0 ) {
110 | instant = Instant.now();
111 | } else {
112 | Object fieldData = kettleRow.getRow()[ fieldIndex ];
113 | if ( ValueMetaInterface.TYPE_TIMESTAMP == fieldValueMeta.getType() ) {
114 | java.sql.Timestamp timestamp = ( (ValueMetaTimestamp) fieldValueMeta ).getTimestamp( fieldData );
115 | instant = new Instant( timestamp.toInstant() );
116 | } else {
117 | Date date = fieldValueMeta.getDate( fieldData );
118 | if (date==null) {
119 | throw new KettleException( "Timestamp field contains a null value, this can't be used to set a timestamp on a bounded/unbounded collection of data" );
120 | }
121 | instant = new Instant( date.getTime() );
122 | }
123 | }
124 | }
125 |
126 | // Pass the row to the process context
127 | //
128 | processContext.outputWithTimestamp( kettleRow, instant );
129 | writtenCounter.inc();
130 |
131 | } catch ( Exception e ) {
132 | errorCounter.inc();
133 | LOG.error( "Error adding timestamp to rows : " + processContext.element() + ", " + e.getMessage() );
134 | throw new RuntimeException( "Error adding timestamp to rows", e );
135 | }
136 | }
137 |
138 | @Override public Duration getAllowedTimestampSkew() {
139 | return Duration.standardMinutes( 120 );
140 | }
141 | }
142 |
--------------------------------------------------------------------------------