-
Notifications
You must be signed in to change notification settings - Fork 3
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
DB-12042 JMS Input Source #32
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,92 @@ | ||
/* | ||
* Copyright 2020 https://github.com/jksinghpro/spark-jms | ||
* Copyright 2021 Splice Machine, Inc. | ||
* | ||
* Licensed under the Apache License, Version 2.0 (the "License"); | ||
* you may not use this file except in compliance with the License. | ||
* You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package com.splicemachine.spark.jms | ||
|
||
import java.util.Properties | ||
import javax.jms.ConnectionFactory | ||
|
||
import com.rabbitmq.jms.admin.RMQConnectionFactory | ||
import io.confluent.kafka.jms.KafkaConnectionFactory | ||
import org.apache.activemq.ActiveMQConnectionFactory | ||
|
||
/** | ||
* Created by exa00015 on 24/12/18. | ||
*/ | ||
trait ConnectionFactoryProvider { | ||
|
||
def createConnection(options:Map[String,String]):ConnectionFactory | ||
|
||
} | ||
|
||
class AMQConnectionFactoryProvider extends ConnectionFactoryProvider { | ||
|
||
override def createConnection(options: Map[String, String]): ConnectionFactory = { | ||
new ActiveMQConnectionFactory | ||
} | ||
} | ||
|
||
class IBMMQConnectionFactoryProvider extends ConnectionFactoryProvider { | ||
|
||
override def createConnection(options: Map[String, String]): ConnectionFactory = { | ||
import com.ibm.msg.client.wmq.common.CommonConstants | ||
import com.ibm.msg.client.jms.JmsConstants | ||
val cf = com.ibm.msg.client.jms.JmsFactoryFactory.getInstance(JmsConstants.WMQ_PROVIDER).createConnectionFactory | ||
cf.setStringProperty(CommonConstants.WMQ_HOST_NAME, options.getOrElse("host", throw new IllegalArgumentException("Option host is required"))) | ||
cf.setIntProperty(CommonConstants.WMQ_PORT, options.getOrElse("port", throw new IllegalArgumentException("Option port is required")).toInt) | ||
cf.setStringProperty(CommonConstants.WMQ_CHANNEL, options.getOrElse("channel", throw new IllegalArgumentException("Option channel is required"))) | ||
cf.setIntProperty(CommonConstants.WMQ_CONNECTION_MODE, CommonConstants.WMQ_CM_CLIENT) | ||
cf.setStringProperty(CommonConstants.WMQ_QUEUE_MANAGER, options.getOrElse("queue_manager", throw new IllegalArgumentException("Option queue_manager is required"))) | ||
options.get("application").foreach( | ||
app => cf.setStringProperty(CommonConstants.WMQ_APPLICATIONNAME, app) | ||
) | ||
cf.setBooleanProperty(JmsConstants.USER_AUTHENTICATION_MQCSP, true) | ||
options.get("userid").foreach( | ||
userid => cf.setStringProperty(JmsConstants.USERID, userid) | ||
) | ||
options.get("password").foreach( | ||
password => cf.setStringProperty(JmsConstants.PASSWORD, password) | ||
) | ||
cf | ||
} | ||
} | ||
|
||
class RMQConnectionFactoryProvider extends ConnectionFactoryProvider { | ||
|
||
override def createConnection(options: Map[String, String]): ConnectionFactory = { | ||
val connectionFactory = new RMQConnectionFactory | ||
connectionFactory.setUsername(options.getOrElse("username",throw new IllegalArgumentException("Option username is required"))) | ||
connectionFactory.setPassword(options.getOrElse("password",throw new IllegalArgumentException("Option password is required"))) | ||
connectionFactory.setVirtualHost(options.getOrElse("virtualhost",throw new IllegalArgumentException("Option virtualhost is required"))) | ||
connectionFactory.setHost(options.getOrElse("host",throw new IllegalArgumentException("Option host is required"))) | ||
connectionFactory.setPort(options.getOrElse("port",throw new IllegalArgumentException("Option port is required")).toInt) | ||
connectionFactory | ||
} | ||
} | ||
|
||
class KafkaConnectionFactoryProvider extends ConnectionFactoryProvider { | ||
|
||
override def createConnection(options: Map[String, String]): ConnectionFactory = { | ||
val props = new Properties | ||
props.put("bootstrap.servers", options.getOrElse("bootstrap.servers",throw new IllegalArgumentException("Option bootstrap.servers is required"))) | ||
props.put("zookeeper.connect", options.getOrElse("zookeeper.connect",throw new IllegalArgumentException("Option zookeeper.connect is required"))) | ||
props.put("client.id", options.getOrElse("client.id",throw new IllegalArgumentException("Option client.id is required" ))) | ||
val connectionFactory = new KafkaConnectionFactory(props) | ||
connectionFactory | ||
} | ||
|
||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,36 @@ | ||
/* | ||
* Copyright 2020 https://github.com/jksinghpro/spark-jms | ||
* | ||
* Licensed under the Apache License, Version 2.0 (the "License"); | ||
* you may not use this file except in compliance with the License. | ||
* You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package com.splicemachine.spark.jms | ||
|
||
import org.apache.spark.rdd.RDD | ||
import org.apache.spark.sql.sources.{BaseRelation, TableScan} | ||
import org.apache.spark.sql.types.StructType | ||
import org.apache.spark.sql.{DataFrame, Row, SQLContext} | ||
|
||
/** | ||
* Created by exa00015 on 24/12/18. | ||
*/ | ||
class DataFrameRelation(override val sqlContext: SQLContext,data:DataFrame) extends BaseRelation with TableScan with Serializable { | ||
|
||
override def schema: StructType = { | ||
data.schema | ||
} | ||
|
||
override def buildScan(): RDD[Row] = { | ||
data.rdd | ||
} | ||
|
||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,96 @@ | ||
/* | ||
* Copyright 2020 https://github.com/jksinghpro/spark-jms | ||
* Copyright 2021 Splice Machine, Inc. | ||
* | ||
* Licensed under the Apache License, Version 2.0 (the "License"); | ||
* you may not use this file except in compliance with the License. | ||
* You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package com.splicemachine.spark.jms | ||
|
||
|
||
import javax.jms.Session | ||
import org.apache.spark.sql.catalyst.ScalaReflection | ||
import org.apache.spark.sql.execution.streaming.{Sink, Source} | ||
import org.apache.spark.sql.{DataFrame, Row, SQLContext, SaveMode} | ||
import org.apache.spark.sql.sources._ | ||
import org.apache.spark.sql.streaming.{DataStreamReader, OutputMode} | ||
import org.apache.spark.sql.types.StructType | ||
import org.apache.spark.sql.jms.JmsStreamingSource | ||
|
||
/** | ||
* Created by exa00015 on 17/12/18. | ||
*/ | ||
private[jms] class DefaultSource extends RelationProvider with SchemaRelationProvider with CreatableRelationProvider with DataSourceRegister with StreamSourceProvider with StreamSinkProvider{ | ||
|
||
override def createRelation(sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { | ||
createRelation(sqlContext, parameters, null) | ||
} | ||
|
||
override def createRelation(sqlContext: SQLContext, parameters: Map[String, String], schema: StructType): BaseRelation = { | ||
parameters.get("queue") match { | ||
case Some(queue) => new JmsDatasourceRelation(sqlContext, parameters) | ||
case None => throw new IllegalArgumentException("Option queue is needed") | ||
} | ||
} | ||
|
||
|
||
override def shortName(): String = "jms" | ||
|
||
override def createRelation(sqlContext: SQLContext, mode: SaveMode, parameters: Map[String, String], data: DataFrame): BaseRelation = { | ||
if (mode == SaveMode.Overwrite) { | ||
throw new UnsupportedOperationException("Data in a kafka topic cannot be overidden !" | ||
+ " Delete topic to implement this functionality") | ||
} | ||
data.foreachPartition((rowIter: Iterator[Row]) => { | ||
val connection = DefaultSource.connectionFactory(parameters).createConnection | ||
val session = connection.createSession(false, Session.AUTO_ACKNOWLEDGE) | ||
val queue = session.createQueue(parameters.getOrElse("queue",throw new IllegalArgumentException("Option queue is required"))) | ||
val producer = session.createProducer(queue) | ||
rowIter.foreach( | ||
record => { | ||
val msg = session.createTextMessage(record.toString()) | ||
producer.send(msg) | ||
}) | ||
producer.close | ||
connection.close | ||
session.close | ||
} | ||
) | ||
new DataFrameRelation(sqlContext, data) | ||
} | ||
|
||
override def sourceSchema(sqlContext: SQLContext, schema: Option[StructType], providerName: String, parameters: Map[String, String]): (String, StructType) = { | ||
require(schema.isEmpty, "JMS source has a fixed schema and cannot be set with a custom one") | ||
(shortName,ScalaReflection.schemaFor[JmsMessage].dataType.asInstanceOf[StructType]) | ||
} | ||
|
||
override def createSource(sqlContext: SQLContext, metadataPath: String, schema: Option[StructType], providerName: String, parameters: Map[String, String]): Source = { | ||
new JmsStreamingSource(sqlContext,parameters, metadataPath, true) | ||
} | ||
|
||
override def createSink(sqlContext: SQLContext, parameters: Map[String, String], partitionColumns: Seq[String], outputMode: OutputMode): Sink = { | ||
new JmsStreamingSink(sqlContext,parameters) | ||
} | ||
} | ||
|
||
|
||
object DefaultSource { | ||
|
||
def connectionFactory(parameters: Map[String, String]) = parameters("connection") match { | ||
case "amq" => new AMQConnectionFactoryProvider().createConnection(parameters) | ||
case "ibmmq" => new IBMMQConnectionFactoryProvider().createConnection(parameters) | ||
case "rmq" => new RMQConnectionFactoryProvider().createConnection(parameters) | ||
case "kafka" => new KafkaConnectionFactoryProvider().createConnection(parameters) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We don't need |
||
case connection: String => Class.forName(connection).newInstance.asInstanceOf[ConnectionFactoryProvider].createConnection(parameters) | ||
} | ||
|
||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,83 @@ | ||
/* | ||
* Copyright 2020 https://github.com/jksinghpro/spark-jms | ||
* | ||
* Licensed under the Apache License, Version 2.0 (the "License"); | ||
* you may not use this file except in compliance with the License. | ||
* You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package com.splicemachine.spark.jms | ||
|
||
import java.util.Collections | ||
import javax.jms._ | ||
|
||
import org.apache.activemq.ActiveMQConnectionFactory | ||
import org.apache.spark.rdd.RDD | ||
import org.apache.spark.sql.catalyst.ScalaReflection | ||
import org.apache.spark.sql.{Row, SQLContext} | ||
import org.apache.spark.sql.sources.{BaseRelation, TableScan} | ||
import org.apache.spark.sql.types.{StringType, StructField, StructType} | ||
|
||
import scala.beans.BeanProperty | ||
import scala.collection.mutable.ListBuffer | ||
|
||
/** | ||
* Created by exa00015 on 17/12/18. | ||
*/ | ||
class JmsDatasourceRelation(override val sqlContext: SQLContext, parameters: Map[String, String]) extends BaseRelation with TableScan with Serializable { | ||
|
||
lazy val RECIEVER_TIMEOUT = parameters.getOrElse("reciever.timeout","3000").toLong | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. A typo in |
||
|
||
|
||
override def schema: StructType = { | ||
ScalaReflection.schemaFor[JmsMessage].dataType.asInstanceOf[StructType] | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What a trick! I think |
||
} | ||
|
||
override def buildScan(): RDD[Row] = { | ||
val connection = DefaultSource.connectionFactory(parameters).createConnection | ||
val session: Session = connection.createSession(false, Session.CLIENT_ACKNOWLEDGE) | ||
connection.start | ||
var messageRdd:RDD[Row] = null | ||
try { | ||
val queue = session.createQueue(parameters("queue")) | ||
val consumer = session.createConsumer(queue) | ||
var break = true | ||
val messageList :ListBuffer[JmsMessage] = ListBuffer() | ||
while (break) { | ||
val textMsg = consumer.receive(RECIEVER_TIMEOUT).asInstanceOf[TextMessage] | ||
/*textMsg match { | ||
case msg:TextMessage => | ||
case msg:BytesMessage => { | ||
var byteData:Array[Byte] = null | ||
byteData = new Array[Byte](msg.getBodyLength.asInstanceOf[Int]) | ||
msg.readBytes(byteData) | ||
} | ||
case msg:ObjectMessage=> msg.getObject | ||
case msg:StreamMessage => | ||
case msg:MapMessage => | ||
}*/ | ||
if(parameters.getOrElse("acknowledge","false").toBoolean && textMsg !=null){ | ||
textMsg.acknowledge | ||
} | ||
textMsg match { | ||
case null => break = false | ||
case _ => messageList += JmsMessage(textMsg) | ||
} | ||
} | ||
import scala.collection.JavaConverters._ | ||
val messageDf = sqlContext.createDataFrame(messageList.toList.asJava,classOf[JmsMessage]) | ||
messageRdd = messageDf.rdd | ||
} finally { | ||
connection.close | ||
} | ||
messageRdd | ||
} | ||
|
||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks suspicious. A downgrade?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It was changed to get in sync with the version of the Splice DB in the environment where this would be running.