You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "grundprinzip (via GitHub)" <gi...@apache.org> on 2023/03/14 12:04:48 UTC

[GitHub] [spark] grundprinzip commented on a diff in pull request #40297: [SPARK-42412][WIP] Initial PR of Spark connect ML

grundprinzip commented on code in PR #40297:
URL: https://github.com/apache/spark/pull/40297#discussion_r1135241667


##########
connector/connect/common/src/main/protobuf/spark/connect/base.proto:
##########
@@ -261,6 +263,9 @@ message ExecutePlanResponse {
     // Special case for executing SQL commands.
     SqlCommandResult sql_command_result = 5;
 
+    // ML command response

Review Comment:
   ```suggestion
       // ML command response.
   ```



##########
connector/connect/common/src/main/protobuf/spark/connect/ml_common.proto:
##########
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+
+syntax = 'proto3';
+
+package spark.connect;
+
+import "spark/connect/expressions.proto";
+
+option java_multiple_files = true;
+option java_package = "org.apache.spark.connect.proto";
+
+
+message MlParams {
+  map<string, Expression.Literal> params = 1;
+  map<string, Expression.Literal> default_params = 2;
+}
+
+message MlStage {
+  string name = 1;
+  MlParams params = 2;
+  string uid = 3;
+  StageType type = 4;
+  enum StageType {

Review Comment:
   Is this knowledge actually required on the client?



##########
connector/connect/common/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -82,13 +83,50 @@ message Relation {
     // Catalog API (experimental / unstable)
     Catalog catalog = 200;
 
+    // ML relation
+    MlRelation ml_relation = 300;
+
     // This field is used to mark extensions to the protocol. When plugins generate arbitrary
     // relations they can add them here. During the planning the correct resolution is done.
     google.protobuf.Any extension = 998;
     Unknown unknown = 999;
   }
 }
 
+message MlRelation {
+  oneof ml_relation_type {
+    ModelTransform model_transform = 1;
+    FeatureTransform feature_transform = 2;
+    ModelAttr model_attr = 3;
+    ModelSummaryAttr model_summary_attr = 4;
+  }
+  message ModelTransform {
+    Relation input = 1;
+    int64 model_ref_id = 2;

Review Comment:
   My suggestion here is to maybe wrap the `moddel_ref_id` into an extra message object that becomes easier to extend.
   
   ```
   message ModelRef {
     int64 id = 1;
   }
   ```
   
   That said, is there a reason the ID is numeric vs a string?



##########
connector/connect/common/src/main/protobuf/spark/connect/ml.proto:
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.
+ */
+
+syntax = 'proto3';
+
+package spark.connect;
+
+import "spark/connect/expressions.proto";
+import "spark/connect/relations.proto";
+import "spark/connect/ml_common.proto";
+
+option java_multiple_files = true;
+option java_package = "org.apache.spark.connect.proto";

Review Comment:
   it would be great to give all of the protos more documentation once we start getting them in.



##########
connector/connect/common/src/main/protobuf/spark/connect/ml_common.proto:
##########
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+
+syntax = 'proto3';
+
+package spark.connect;
+
+import "spark/connect/expressions.proto";
+
+option java_multiple_files = true;
+option java_package = "org.apache.spark.connect.proto";
+
+
+message MlParams {
+  map<string, Expression.Literal> params = 1;
+  map<string, Expression.Literal> default_params = 2;
+}
+
+message MlStage {
+  string name = 1;
+  MlParams params = 2;
+  string uid = 3;
+  StageType type = 4;
+  enum StageType {
+    ESTIMATOR = 0;
+    TRANSFORMER = 1;
+  }

Review Comment:
   In proto the first parameter should allways be `unspecified`. Please follow the style guide https://protobuf.dev/programming-guides/style/#enums



##########
connector/connect/common/src/main/protobuf/spark/connect/ml.proto:
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.
+ */
+
+syntax = 'proto3';
+
+package spark.connect;
+
+import "spark/connect/expressions.proto";
+import "spark/connect/relations.proto";
+import "spark/connect/ml_common.proto";
+
+option java_multiple_files = true;
+option java_package = "org.apache.spark.connect.proto";
+
+
+message MlEvaluator {
+  string name = 1;
+  MlParams params = 2;
+  string uid = 3;
+}
+
+
+message MlCommand {
+  oneof ml_command_type {
+    Fit fit = 1;
+    FetchModelAttr fetch_model_attr = 2;
+    FetchModelSummaryAttr fetch_model_summary_attr = 3;
+    LoadModel load_model = 4;
+    SaveModel save_model = 5;
+    Evaluate evaluate = 6;
+    SaveStage save_stage = 7;
+    LoadStage load_stage = 8;
+    SaveEvaluator save_evaluator = 9;
+    LoadEvaluator load_evaluator = 10;
+    CopyModel copy_model = 11;
+    DeleteModel delete_model = 12;
+  }
+
+  message Fit {
+    MlStage estimator = 1;
+    Relation dataset = 2;
+  }
+
+  message Evaluate {
+    MlEvaluator evaluator = 1;
+  }
+
+  message LoadModel {

Review Comment:
   Would this work with arbitrary model for example provided by Spark NLP?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org