You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@bahir.apache.org by yanlin-Lynn <gi...@git.apache.org> on 2018/12/02 03:04:30 UTC

[GitHub] bahir pull request #72: [BAHIR-183]Using HDFS for saving message for mqtt so...

GitHub user yanlin-Lynn opened a pull request:

    https://github.com/apache/bahir/pull/72

    [BAHIR-183]Using HDFS for saving message for mqtt source.

    Currently in spark-sql-streaming-mqtt, the received mqtt message is saved in a local file by driver, this will have the risks of losing data for cluster mode when application master failover occurs. So add a hdfs-based mqtt source to solve this problem.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/yanlin-Lynn/bahir bahir-183

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/bahir/pull/72.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #72
    
----
commit eb33741f8b77828815c8f834ec0951d6c39728fc
Author: wangyanlin01 <wa...@...>
Date:   2018-12-02T03:00:21Z

    [BAHIR-183]Using HDFS for saving message for mqtt source.

----


---

[GitHub] bahir issue #72: [BAHIR-183]Using HDFS for saving message for mqtt source.

Posted by lresende <gi...@git.apache.org>.
Github user lresende commented on the issue:

    https://github.com/apache/bahir/pull/72
  
    @yanlin-Lynn Please rebase to latest master so I can merge.


---

[GitHub] bahir pull request #72: [BAHIR-183]Using HDFS for saving message for mqtt so...

Posted by lukasz-antoniak <gi...@git.apache.org>.
Github user lukasz-antoniak commented on a diff in the pull request:

    https://github.com/apache/bahir/pull/72#discussion_r238359229
  
    --- Diff: sql-streaming-mqtt/src/main/scala/org/apache/bahir/sql/streaming/mqtt/HDFSMQTTSourceProvider.scala ---
    @@ -0,0 +1,184 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.bahir.sql.streaming.mqtt
    +
    +import java.util.Locale
    +
    +import org.eclipse.paho.client.mqttv3.{MqttClient, MqttConnectOptions}
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.SQLContext
    +import org.apache.spark.sql.execution.streaming.Source
    +import org.apache.spark.sql.sources.{DataSourceRegister, StreamSourceProvider}
    +import org.apache.spark.sql.types.{LongType, StringType, StructField, StructType}
    +
    +import org.apache.bahir.utils.MQTTConfig
    +
    +/**
    + * The provider class for creating MQTT source.
    + * This provider throw IllegalArgumentException if  'brokerUrl' or 'topic' parameter
    + * is not set in options.
    + */
    +class HDFSMQTTSourceProvider extends StreamSourceProvider with DataSourceRegister with Logging {
    +
    +  override def sourceSchema(sqlContext: SQLContext, schema: Option[StructType],
    +    providerName: String, parameters: Map[String, String]): (String, StructType) = {
    +    ("mqtt", HDFSMQTTSourceProvider.SCHEMA_DEFAULT)
    +  }
    +
    +  override def createSource(sqlContext: SQLContext, metadataPath: String,
    +    schema: Option[StructType], providerName: String, parameters: Map[String, String]): Source = {
    +
    +    def e(s: String) = new IllegalArgumentException(s)
    +
    +    val caseInsensitiveParameter = parameters.map{case (key: String, value: String) =>
    +      key.toLowerCase(Locale.ROOT) -> value
    +    }
    +
    +    val brokerUrl: String = getParameterValue(caseInsensitiveParameter, MQTTConfig.brokerUrl, "")
    --- End diff --
    
    We already have a common function to parse configuration parameters, so I would try to reuse/adopt it.


---

[GitHub] bahir pull request #72: [BAHIR-183]Using HDFS for saving message for mqtt so...

Posted by yanlin-Lynn <gi...@git.apache.org>.
Github user yanlin-Lynn closed the pull request at:

    https://github.com/apache/bahir/pull/72


---

[GitHub] bahir issue #72: [BAHIR-183]Using HDFS for saving message for mqtt source.

Posted by lukasz-antoniak <gi...@git.apache.org>.
Github user lukasz-antoniak commented on the issue:

    https://github.com/apache/bahir/pull/72
  
    Would it be possible to implement integration tests with embedded HDFS just to make sure we do not run into regression issues?


---

[GitHub] bahir issue #72: [BAHIR-183]Using HDFS for saving message for mqtt source.

Posted by yanlin-Lynn <gi...@git.apache.org>.
Github user yanlin-Lynn commented on the issue:

    https://github.com/apache/bahir/pull/72
  
    I have done some wrong operation, close this patch and I will pull a new patch again.


---

[GitHub] bahir pull request #72: [BAHIR-183]Using HDFS for saving message for mqtt so...

Posted by yanlin-Lynn <gi...@git.apache.org>.
Github user yanlin-Lynn commented on a diff in the pull request:

    https://github.com/apache/bahir/pull/72#discussion_r238509713
  
    --- Diff: sql-streaming-mqtt/src/main/scala/org/apache/bahir/sql/streaming/mqtt/HDFSMQTTSourceProvider.scala ---
    @@ -0,0 +1,184 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.bahir.sql.streaming.mqtt
    +
    +import java.util.Locale
    +
    +import org.eclipse.paho.client.mqttv3.{MqttClient, MqttConnectOptions}
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.SQLContext
    +import org.apache.spark.sql.execution.streaming.Source
    +import org.apache.spark.sql.sources.{DataSourceRegister, StreamSourceProvider}
    +import org.apache.spark.sql.types.{LongType, StringType, StructField, StructType}
    +
    +import org.apache.bahir.utils.MQTTConfig
    +
    +/**
    + * The provider class for creating MQTT source.
    + * This provider throw IllegalArgumentException if  'brokerUrl' or 'topic' parameter
    + * is not set in options.
    + */
    +class HDFSMQTTSourceProvider extends StreamSourceProvider with DataSourceRegister with Logging {
    +
    +  override def sourceSchema(sqlContext: SQLContext, schema: Option[StructType],
    +    providerName: String, parameters: Map[String, String]): (String, StructType) = {
    +    ("mqtt", HDFSMQTTSourceProvider.SCHEMA_DEFAULT)
    +  }
    +
    +  override def createSource(sqlContext: SQLContext, metadataPath: String,
    +    schema: Option[StructType], providerName: String, parameters: Map[String, String]): Source = {
    +
    +    def e(s: String) = new IllegalArgumentException(s)
    +
    +    val caseInsensitiveParameter = parameters.map{case (key: String, value: String) =>
    +      key.toLowerCase(Locale.ROOT) -> value
    +    }
    +
    +    val brokerUrl: String = getParameterValue(caseInsensitiveParameter, MQTTConfig.brokerUrl, "")
    --- End diff --
    
    Thansk for your time for reviewing. This code is the same as we use in company, I will adjust it later!


---

[GitHub] bahir pull request #72: [BAHIR-183]Using HDFS for saving message for mqtt so...

Posted by lukasz-antoniak <gi...@git.apache.org>.
Github user lukasz-antoniak commented on a diff in the pull request:

    https://github.com/apache/bahir/pull/72#discussion_r238359914
  
    --- Diff: sql-streaming-mqtt/src/main/scala/org/apache/bahir/sql/streaming/mqtt/HDFSMQTTSourceProvider.scala ---
    @@ -0,0 +1,184 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.bahir.sql.streaming.mqtt
    +
    +import java.util.Locale
    +
    +import org.eclipse.paho.client.mqttv3.{MqttClient, MqttConnectOptions}
    +
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.sql.SQLContext
    +import org.apache.spark.sql.execution.streaming.Source
    +import org.apache.spark.sql.sources.{DataSourceRegister, StreamSourceProvider}
    +import org.apache.spark.sql.types.{LongType, StringType, StructField, StructType}
    +
    +import org.apache.bahir.utils.MQTTConfig
    +
    +/**
    + * The provider class for creating MQTT source.
    + * This provider throw IllegalArgumentException if  'brokerUrl' or 'topic' parameter
    + * is not set in options.
    + */
    +class HDFSMQTTSourceProvider extends StreamSourceProvider with DataSourceRegister with Logging {
    +
    +  override def sourceSchema(sqlContext: SQLContext, schema: Option[StructType],
    +    providerName: String, parameters: Map[String, String]): (String, StructType) = {
    +    ("mqtt", HDFSMQTTSourceProvider.SCHEMA_DEFAULT)
    +  }
    +
    +  override def createSource(sqlContext: SQLContext, metadataPath: String,
    +    schema: Option[StructType], providerName: String, parameters: Map[String, String]): Source = {
    +
    +    def e(s: String) = new IllegalArgumentException(s)
    +
    +    val caseInsensitiveParameter = parameters.map{case (key: String, value: String) =>
    +      key.toLowerCase(Locale.ROOT) -> value
    +    }
    +
    +    val brokerUrl: String = getParameterValue(caseInsensitiveParameter, MQTTConfig.brokerUrl, "")
    +
    +    if (brokerUrl.isEmpty) {
    +      throw e("Please specify a brokerUrl, by .options(\"brokerUrl\",...)")
    +    }
    +    logInfo(s"Using brokerUrl $brokerUrl")
    +
    +    // if default is subscribe everything, it leads to getting a lot of unwanted system messages.
    +    val topic: String = getParameterValue(caseInsensitiveParameter, MQTTConfig.topic, "")
    +    if (topic.isEmpty) {
    +      throw e("Please specify a topic, by .options(\"topic\",...)")
    +    }
    +    logInfo(s"Subscribe topic $topic")
    +
    +    val clientId: String = getParameterValue(caseInsensitiveParameter, MQTTConfig.clientId, {
    +      val randomClientId = MqttClient.generateClientId()
    +      logInfo(s"Using random clientId ${randomClientId}.")
    +      randomClientId
    +    })
    +    logInfo(s"ClientId $clientId")
    +
    +    val usernameString: String = getParameterValue(caseInsensitiveParameter,
    +      MQTTConfig.username, "")
    +    val username: Option[String] = if (usernameString.isEmpty) {
    +      None
    +    } else {
    +      Some(usernameString)
    +    }
    +
    +    val passwordString: String = getParameterValue(caseInsensitiveParameter,
    +      MQTTConfig.password, "")
    +    val password: Option[String] = if (passwordString.isEmpty) {
    +      None
    +    } else {
    +      Some(passwordString)
    +    }
    +
    +    val connectionTimeout: Int = getParameterValue(
    +      caseInsensitiveParameter,
    +      MQTTConfig.connectionTimeout,
    +      MqttConnectOptions.CONNECTION_TIMEOUT_DEFAULT.toString
    +    ).toInt
    +    logInfo(s"Set connection timeout $connectionTimeout")
    +
    +    val keepAlive: Int = getParameterValue(
    +      caseInsensitiveParameter,
    +      MQTTConfig.keetAliveInterval,
    +      MqttConnectOptions.KEEP_ALIVE_INTERVAL_DEFAULT.toString
    +    ).toInt
    +    logInfo(s"Set keep alive interval $keepAlive")
    +
    +    val mqttVersion: Int = getParameterValue(
    +      caseInsensitiveParameter,
    +      MQTTConfig.mqttVersion,
    +      MqttConnectOptions.MQTT_VERSION_DEFAULT.toString
    +    ).toInt
    +    logInfo(s"Set mqtt version $mqttVersion")
    +
    +    val cleanSession: Boolean = getParameterValue(
    +      caseInsensitiveParameter,
    +      MQTTConfig.cleanSession,
    +      "true"
    +    ).toBoolean
    +    logInfo(s"Set clean session $cleanSession")
    +
    +    val qos: Int = getParameterValue(
    +      caseInsensitiveParameter,
    +      MQTTConfig.qos,
    +      "0"
    +    ).toInt
    +    logInfo(s"Set qos $qos")
    +
    +    val maxBatchMessageNum = getParameterValue(
    +      caseInsensitiveParameter,
    +      MQTTConfig.maxBatchMessageNum, s"${Long.MaxValue}"
    +    ).toLong
    +    logInfo(s"Control max message number in one batch $maxBatchMessageNum")
    +
    +    val maxBatchMessageSize = getParameterValue(
    +      caseInsensitiveParameter,
    +      MQTTConfig.maxBatchMessageSize, s"${Long.MaxValue}"
    +    ).toLong
    +    logInfo(s"Control max message content size in one batch $maxBatchMessageSize")
    +
    +    val maxRetryNumber = getParameterValue(
    +      caseInsensitiveParameter,
    +      MQTTConfig.maxRetryNumber, "3"
    +    ).toInt
    +    logInfo(s"Set max retry number $maxRetryNumber")
    +
    +    val mqttConnectOptions: MqttConnectOptions = new MqttConnectOptions()
    +    mqttConnectOptions.setAutomaticReconnect(true)
    +    mqttConnectOptions.setCleanSession(cleanSession)
    +    mqttConnectOptions.setConnectionTimeout(connectionTimeout)
    +    mqttConnectOptions.setKeepAliveInterval(keepAlive)
    +    mqttConnectOptions.setMqttVersion(mqttVersion)
    +    (username, password) match {
    +      case (Some(u: String), Some(p: String)) =>
    +        mqttConnectOptions.setUserName(u)
    +        mqttConnectOptions.setPassword(p.toCharArray)
    +      case _ =>
    +    }
    +
    +    new HdfsBasedMQTTStreamSource(
    +      sqlContext,
    +      metadataPath,
    +      brokerUrl,
    +      topic,
    +      clientId,
    +      mqttConnectOptions,
    +      qos,
    +      maxBatchMessageNum,
    +      maxBatchMessageSize,
    +      maxRetryNumber)
    +  }
    +
    +  override def shortName(): String = "hdfs-mqtt"
    +
    +  private def getParameterValue(
    +      parameters: Map[String, String],
    +      parameterName: String,
    +      defaultValue: String): String = {
    +
    +    parameters.getOrElse(parameterName.toLowerCase(Locale.ROOT), defaultValue)
    +  }
    +}
    +
    +object HDFSMQTTSourceProvider {
    +  val SCHEMA_DEFAULT = StructType(
    --- End diff --
    
    There is already a schema for MQTT messages in `MQTTStreamConstants` object. Why not reuse it?


---