You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@dolphinscheduler.apache.org by GitBox <gi...@apache.org> on 2022/08/27 06:52:47 UTC

[GitHub] [dolphinscheduler] reele opened a new pull request, #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

reele opened a new pull request, #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677

   <!--Thanks very much for contributing to Apache DolphinScheduler. Please review https://dolphinscheduler.apache.org/en-us/community/development/pull-request.html before opening a pull request.-->
   
   
   ## Purpose of the pull request
   
   ```xml
       <select id="queryLastTaskInstance" resultType="org.apache.dolphinscheduler.dao.entity.TaskInstance">
           select
           <include refid="baseSql"/>
           from t_ds_task_instance
           where task_code=#{taskCode}
           <if test="startTime!=null and endTime != null">
               and start_time <![CDATA[ >= ]]> #{startTime} and start_time <![CDATA[ <= ]]> #{endTime}
           </if>
           order by end_time desc limit 1
       </select>
       <select id="queryLastTaskInstanceList" resultType="org.apache.dolphinscheduler.dao.entity.TaskInstance">
           select
           <include refid="baseSql"/>
           from t_ds_task_instance
           where 1=1
           <if test="taskCodes != null and taskCodes.size() != 0">
               and task_code in
               <foreach collection="taskCodes" index="index" item="i" open="(" separator="," close=")">
                   #{i}
               </foreach>
           </if>
           <if test="startTime!=null and endTime != null">
               and start_time <![CDATA[ >= ]]> #{startTime} and start_time <![CDATA[ <= ]]> #{endTime}
           </if>
       </select>
   ```
   Now v2.0.6 and v2.0.7 dependent task use `start_time` to find the target task above, when the task is blocked by preview tasks and started in after days, the dependent task will never succeed.
   
   The same problem also occurs in `complement` tasks.
   
   ## Brief change log
   
   Use `process_instance_id` instead of `start_time`, and also follows the process_instance's `schedule_time`.
   


-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] JinyLeeChina commented on pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
JinyLeeChina commented on PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#issuecomment-1233640878

   Which issue does this PR repair ?


-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r966805881


##########
dolphinscheduler-dao/src/main/resources/sql/upgrade/2.0.7_schema/postgresql/dolphinscheduler_ddl.sql:
##########
@@ -0,0 +1,20 @@
+/*
+ * 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.
+*/
+
+drop index if exists task_instance_idx_code_pid_edtm;
+create index task_instance_idx_code_pid_edtm on t_ds_task_instance (task_code, process_instance_id, end_time);

Review Comment:
   ok.



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] JinyLeeChina commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix #11725 dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
JinyLeeChina commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r975308012


##########
dolphinscheduler-dao/src/main/resources/sql/upgrade/2.0.7_schema/postgresql/dolphinscheduler_ddl.sql:
##########
@@ -0,0 +1,20 @@
+/*
+ * 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.
+*/
+
+drop index if exists task_instance_idx_code_pid_edtm;
+create index task_instance_idx_code_pid_edtm on t_ds_task_instance (task_code, process_instance_id, end_time);

Review Comment:
   OK, thanks



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r965682277


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java:
##########
@@ -81,7 +81,7 @@ IPage<TaskInstance> queryTaskInstanceListPaging(IPage<TaskInstance> page,
      * @param endTime endTime
      * @return task instance
      */
-    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("startTime") Date startTime, @Param("endTime") Date endTime);
+    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("processInstanceId") long processInstanceId);

Review Comment:
   because taskInstance don't keep the `scheduleTime`, but processInstance do, if only use startTime, and processInstance start with a scheduleTime, the dependent node will use `scheduleTime` to calculate a `bizdate` to find the task, it may can't detect the task in original `bizdate`.
   
   the actual situation I have encountered is:
   etl task `B` processes a zipper table, it must run continuously by date, it's previews dependent task `D`  depends on `A[T]` and `B[T-1](self yesterday)`, in some reason, `A` blocked until the `next day`, so `B.startTime` is `[T+1]`, and next day `D` will use `bizdate[T]` calculated by `scheduleTime[T+1]`  to finding `B[T]`, there is no `B` started on time `[T]`, so the dependent task `D[T+1]` will never stop, every time this happens, I need to manually stop the processInstance, and set the state of `D` to `Forced Success`, and recover the process.
   
   In fact taskInstance can only run attached to a processInstance, and taskInstance's system.biz.date parameter is same as processInstance, so I think it is right to find taskInstance through processInstance, and in 2.0.5, it did in the same way.



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r965682277


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java:
##########
@@ -81,7 +81,7 @@ IPage<TaskInstance> queryTaskInstanceListPaging(IPage<TaskInstance> page,
      * @param endTime endTime
      * @return task instance
      */
-    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("startTime") Date startTime, @Param("endTime") Date endTime);
+    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("processInstanceId") long processInstanceId);

Review Comment:
   because taskInstance don't keep the `scheduleTime`, but processInstance do, if only use start_time, and processInstance start with a scheduleTime, the dependent node will use `scheduleTime` to calculate a `bizdate` to find the task, it may can't detect the task in original `bizdate`.
   
   the actual situation I have encountered is:
   etl task `B` processes a zipper table, it must run continuously by date, it's previews dependent task `D`  depends on `A[T]` and `B[T-1](self yesterday)`, in some reason, `A` blocked until the `next day`, so `B.startTime` is `[T+1]`, and next day `D` will use `bizdate[T]` calculated by `scheduleTime[T+1]`  to finding `B[T]`, there is no `B` started on time `[T]`, so the dependent task `D[T+1]` will never stop, every time this happens, I need to manually stop the processInstance, and set the state of `D` to `Forced Success`, and recover the process.
   
   In fact taskInstance can only run attached to a processInstance, so I think it is right to find taskInstance through processInstance, and in 2.0.5, it did in the same way.



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] JinyLeeChina commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
JinyLeeChina commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r966814191


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java:
##########
@@ -81,7 +81,7 @@ IPage<TaskInstance> queryTaskInstanceListPaging(IPage<TaskInstance> page,
      * @param endTime endTime
      * @return task instance
      */
-    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("startTime") Date startTime, @Param("endTime") Date endTime);
+    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("processInstanceId") long processInstanceId);

Review Comment:
   I see what you mean. I think there were bugs before 2.0.5. The execution of tasks in the time cycle has nothing to do with which workflow instance. A task should be finished yesterday, but it finish in today, is there a problem ? And it should be solved manually



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r965712336


##########
dolphinscheduler-dao/src/main/resources/sql/upgrade/2.0.7_schema/postgresql/dolphinscheduler_ddl.sql:
##########
@@ -0,0 +1,20 @@
+/*
+ * 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.
+*/
+
+drop index if exists task_instance_idx_code_pid_edtm;
+create index task_instance_idx_code_pid_edtm on t_ds_task_instance (task_code, process_instance_id, end_time);

Review Comment:
   it's a performance optimize if use this pr, on my platform, it have 5000+ dependent tasks to run, and generated 15000 records every day in t_ds_task_instance, in preview version, I added index (task_code, end_time)(2.0.6) and (process_instance_id, flag, start_time)(<=2.0.5), if don't create this index, concurrent dependent tasks will perform a large number of full table scans on t_ds_task_instance, resulting in huge resource consumption and even database paralysis,this problem occurs in my database(stand-alone server with 8c 16GB linux, pg9.7), so I have to add this index to fix that.



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r965712336


##########
dolphinscheduler-dao/src/main/resources/sql/upgrade/2.0.7_schema/postgresql/dolphinscheduler_ddl.sql:
##########
@@ -0,0 +1,20 @@
+/*
+ * 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.
+*/
+
+drop index if exists task_instance_idx_code_pid_edtm;
+create index task_instance_idx_code_pid_edtm on t_ds_task_instance (task_code, process_instance_id, end_time);

Review Comment:
   it's a performance optimize if use this pr, on my platform, it have 5000+ dependent task to run, and generated 15000 records every day in t_ds_task_instance, in preview version, I added index (task_code, end_time)(2.0.6) and (process_instance_id, flag, start_time)(<=2.0.5), if don't create this index, a large number of full table scans will lead to huge resource consumption on scheduler's database which will be paralyzed.



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] JinyLeeChina commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix #11725 dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
JinyLeeChina commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r975279892


##########
dolphinscheduler-dao/src/main/resources/sql/upgrade/2.0.7_schema/postgresql/dolphinscheduler_ddl.sql:
##########
@@ -0,0 +1,20 @@
+/*
+ * 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.
+*/
+
+drop index if exists task_instance_idx_code_pid_edtm;
+create index task_instance_idx_code_pid_edtm on t_ds_task_instance (task_code, process_instance_id, end_time);

Review Comment:
   which is new pr ?



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix #11725 dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r975432715


##########
dolphinscheduler-dao/src/main/resources/sql/upgrade/2.0.7_schema/postgresql/dolphinscheduler_ddl.sql:
##########
@@ -0,0 +1,20 @@
+/*
+ * 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.
+*/
+
+drop index if exists task_instance_idx_code_pid_edtm;
+create index task_instance_idx_code_pid_edtm on t_ds_task_instance (task_code, process_instance_id, end_time);

Review Comment:
   I created a pr before, but not merged, no reason, it was auto closed:
   #8290



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#issuecomment-1233649542

   > Which issue does this PR repair ?
   
   I added a pr:#11725


-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] JinyLeeChina commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
JinyLeeChina commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r966804423


##########
dolphinscheduler-dao/src/main/resources/sql/upgrade/2.0.7_schema/postgresql/dolphinscheduler_ddl.sql:
##########
@@ -0,0 +1,20 @@
+/*
+ * 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.
+*/
+
+drop index if exists task_instance_idx_code_pid_edtm;
+create index task_instance_idx_code_pid_edtm on t_ds_task_instance (task_code, process_instance_id, end_time);

Review Comment:
   I agree with you, this index is important, but it has nothing to do with the current issue, please submit PR separately



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


Re: [PR] [Fix][2.0.6-2.0.7] Fix #11725 dependent task cannot detect target task finished across days [dolphinscheduler]

Posted by "SbloodyS (via GitHub)" <gi...@apache.org>.
SbloodyS commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r1417106870


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java:
##########
@@ -81,7 +81,7 @@ IPage<TaskInstance> queryTaskInstanceListPaging(IPage<TaskInstance> page,
      * @param endTime endTime
      * @return task instance
      */
-    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("startTime") Date startTime, @Param("endTime") Date endTime);
+    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("processInstanceId") long processInstanceId);

Review Comment:
   > Yes, I understand. usually data processing is required to be completed on the same day, but there are other tasks that need to be performed continuously every day and not sensitive to date delays. upstream data may have acceptable latency, when the delayed data is released, we usually expect the scheduling platform to automatically and continuously process the delayed data (including tasks that are blocked downstream for many days), especially when some third-party data access is involved.
   > 
   > Some scheduling platforms have the feature of automatic date flop of tasks, especially this feature can also be implemented through the `dependent task` of Dolphinscheduler, `system.biz.date` parameter can make sure the task can get the correct date, dependent task(depend self yesterday) can make sure the task runs in date continuously.
   > 
   > and most important is, this update changed some rules, that can never use `complement` command to rerun the delayed process which include `dependent task`(it never worked with complement).
   
   I'm +1 on this. Would you like to resubmit this pr to 2.0.9-prepare branch?



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix #11725 dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r975301972


##########
dolphinscheduler-dao/src/main/resources/sql/upgrade/2.0.7_schema/postgresql/dolphinscheduler_ddl.sql:
##########
@@ -0,0 +1,20 @@
+/*
+ * 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.
+*/
+
+drop index if exists task_instance_idx_code_pid_edtm;
+create index task_instance_idx_code_pid_edtm on t_ds_task_instance (task_code, process_instance_id, end_time);

Review Comment:
   not create yet...
   because... this index is only valid for this PR, if create alone, it has no effect on versions 2.0.0 to 2.0.6...



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r966887972


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java:
##########
@@ -81,7 +81,7 @@ IPage<TaskInstance> queryTaskInstanceListPaging(IPage<TaskInstance> page,
      * @param endTime endTime
      * @return task instance
      */
-    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("startTime") Date startTime, @Param("endTime") Date endTime);
+    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("processInstanceId") long processInstanceId);

Review Comment:
   Yes, I understand.
   usually data processing is required to be completed on the same day,
   but there are other tasks that need to be performed continuously every day and not sensitive to date delays.
   upstream data may have acceptable latency, when the delayed data is released,
   we usually expect the scheduling platform to automatically and continuously process the delayed data (including tasks that are blocked downstream for many days), especially when some third-party data access is involved.
   
   Some scheduling platforms have the feature of automatic date flop of tasks, especially this feature can also be implemented through the `dependent task` of Dolphinscheduler.
   
   most important is, this update changed some rules, i can never use `complement` command to rerun the delayed process which include `dependent task`(it never worked with complement), that's why i submited this pr.



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r966887972


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java:
##########
@@ -81,7 +81,7 @@ IPage<TaskInstance> queryTaskInstanceListPaging(IPage<TaskInstance> page,
      * @param endTime endTime
      * @return task instance
      */
-    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("startTime") Date startTime, @Param("endTime") Date endTime);
+    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("processInstanceId") long processInstanceId);

Review Comment:
   Yes, I understand.
   usually data processing is required to be completed on the same day,
   but there are other tasks that need to be performed continuously every day and not sensitive to date delays.
   upstream data may have acceptable latency, when the delayed data is released,
   we usually expect the scheduling platform to automatically and continuously process the delayed data (including tasks that are blocked downstream for many days), especially when some third-party data access is involved.
   
   Some scheduling platforms have the feature of automatic date flop of tasks, especially this feature can also be implemented through the `dependent task` of Dolphinscheduler.
   
   most important is, this update changed some rules, i can never use `complement` command to rerun the delayed process which include `dependent task`(it never worked with complement), that why i submited this pr.



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r966887972


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java:
##########
@@ -81,7 +81,7 @@ IPage<TaskInstance> queryTaskInstanceListPaging(IPage<TaskInstance> page,
      * @param endTime endTime
      * @return task instance
      */
-    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("startTime") Date startTime, @Param("endTime") Date endTime);
+    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("processInstanceId") long processInstanceId);

Review Comment:
   Yes, I understand.
   usually data processing is required to be completed on the same day,
   but there are other tasks that need to be performed continuously every day and not sensitive to date delays.
   upstream data may have acceptable latency, when the delayed data is released,
   we usually expect the scheduling platform to automatically and continuously process the delayed data (including tasks that are blocked downstream for many days), especially when some third-party data access is involved.
   
   Some scheduling platforms have the feature of automatic date flop of tasks, especially this feature can also be implemented through the `dependent task` of Dolphinscheduler, `system.biz.date` parameter can make sure the task can get the correct date, dependent task(depend self yesterday) can make sure the task runs in date continuously.
   
   and most important is, this update changed some rules, that can never use `complement` command to rerun the delayed process which include `dependent task`(it never worked with complement).



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r965682277


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java:
##########
@@ -81,7 +81,7 @@ IPage<TaskInstance> queryTaskInstanceListPaging(IPage<TaskInstance> page,
      * @param endTime endTime
      * @return task instance
      */
-    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("startTime") Date startTime, @Param("endTime") Date endTime);
+    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("processInstanceId") long processInstanceId);

Review Comment:
   because taskInstance don't keep the `scheduleTime`, but processInstance do, if only use startTime, and processInstance start with a scheduleTime, the dependent node will use `scheduleTime` to find the task, it may can't detect the task in original `scheduleTime`.
   
   the actual situation I have encountered is:
   etl task `B` processes a zipper table, it must run continuously by date, and used the parameter `system.biz.date`, it's previews dependent task `D`  depends on `A[T]` and `B[T-1](self yesterday)`, in some reason, `A` blocked until the `next day`, so `B.startTime` is `[T+1]`, and next day `D` will use `scheduleTime[T]` calculated by `scheduleTime[T+1]`  to finding `B[T]`, there is no `B` started on time `[T]`, so the dependent task `D[T+1]` will never stop, every time this happens, I need to manually stop the processInstance, and set the state of `D` to `Forced Success`, and recover the process.
   
   In fact taskInstance can only run attached to a processInstance, and taskInstance's system.biz.date parameter is same as processInstance, so I think it is right to find taskInstance through processInstance, and in 2.0.5, it did in the same way.



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r965682277


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java:
##########
@@ -81,7 +81,7 @@ IPage<TaskInstance> queryTaskInstanceListPaging(IPage<TaskInstance> page,
      * @param endTime endTime
      * @return task instance
      */
-    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("startTime") Date startTime, @Param("endTime") Date endTime);
+    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("processInstanceId") long processInstanceId);

Review Comment:
   because taskInstance don't keep the `scheduleTime`, but processInstance do, if only use start_time, and processInstance start with a scheduleTime, the dependent node will use `scheduleTime` to calculate a `bizdate` to find the task, it may can't detect the task in original `bizdate`.
   
   the actual situation I have encountered is:
   etl task `B` processes a zipper table, it must run continuously by date, it's previews dependent task `D`  depends on `A[T]` and `B[T-1](self yesterday)`, in some reason, `A` blocked until the `next day`, so `B.startTime` is `[T+1]`, and next day `D` will use `bizdate[T]` calculated by `scheduleTime[T+1]`  to finding `B[T]`, there is no `B` started on time `[T]`, that makes 
    dependent task `D[T+1]` will never stop, every time this happens, I need to manually stop the processInstance, and set the state of `D` to `Forced Success`.
   
   In fact taskInstance can only run attached to a processInstance, so I think it is right to find taskInstance through processInstance, and in 2.0.5, it did in the same way.



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r965682277


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java:
##########
@@ -81,7 +81,7 @@ IPage<TaskInstance> queryTaskInstanceListPaging(IPage<TaskInstance> page,
      * @param endTime endTime
      * @return task instance
      */
-    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("startTime") Date startTime, @Param("endTime") Date endTime);
+    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("processInstanceId") long processInstanceId);

Review Comment:
   because taskInstance don't keep the `scheduleTime`, but processInstance do, if only use start_time, and processInstance start with a scheduleTime, the dependent node will use `scheduleTime` to calculate a `bizdate` to find the task, it may can't detect the task in original `bizdate`.
   
   the actual situation I have encountered is:
   etl task `B` processes a zipper table, it must run continuously by date, it's previews dependent task `D`  depends on `A[T]` and `B[T-1](self yesterday)`, in some reason, `A` blocked until the `next day`, so `B.startTime` is `[T+1]`, and next day `D` will use `bizdate[T]` calculated by `scheduleTime[T+1]`  to finding `B[T]`, there is no `B` started on time `[T]`, so the dependent task `D[T+1]` will never stop, every time this happens, I need to manually stop the processInstance, and set the state of `D` to `Forced Success`.
   
   In fact taskInstance can only run attached to a processInstance, so I think it is right to find taskInstance through processInstance, and in 2.0.5, it did in the same way.



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r965712336


##########
dolphinscheduler-dao/src/main/resources/sql/upgrade/2.0.7_schema/postgresql/dolphinscheduler_ddl.sql:
##########
@@ -0,0 +1,20 @@
+/*
+ * 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.
+*/
+
+drop index if exists task_instance_idx_code_pid_edtm;
+create index task_instance_idx_code_pid_edtm on t_ds_task_instance (task_code, process_instance_id, end_time);

Review Comment:
   it's a performance optimize if use this pr, on my platform, it have 5000+ dependent task to run, and generated 15000 records every day in t_ds_task_instance, in preview version, I added index (task_code, end_time)(2.0.6) and (process_instance_id, flag, start_time)(<=2.0.5), if don't create this index, concurrent dependent tasks will perform a large number of full table scans on t_ds_task_instance, resulting in huge resource consumption and even database paralysis,this problem occurs in my database(stand-alone server with 8c 16GB linux, pg9.7), so I have to add this index to fix that.



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r965712336


##########
dolphinscheduler-dao/src/main/resources/sql/upgrade/2.0.7_schema/postgresql/dolphinscheduler_ddl.sql:
##########
@@ -0,0 +1,20 @@
+/*
+ * 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.
+*/
+
+drop index if exists task_instance_idx_code_pid_edtm;
+create index task_instance_idx_code_pid_edtm on t_ds_task_instance (task_code, process_instance_id, end_time);

Review Comment:
   it's a performance optimize if use this pr, on my platform, it have 5000+ dependent task to run, and generated 15000 records every day in t_ds_task_instance, in preview version, I added index (task_code, end_time)(2.0.6) and (process_instance_id, flag, start_time)(<=2.0.5), if don't create this index, concurrent dependent tasks will perform a large number of full table scans, resulting in huge resource consumption and even database paralysis.



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r966887972


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java:
##########
@@ -81,7 +81,7 @@ IPage<TaskInstance> queryTaskInstanceListPaging(IPage<TaskInstance> page,
      * @param endTime endTime
      * @return task instance
      */
-    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("startTime") Date startTime, @Param("endTime") Date endTime);
+    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("processInstanceId") long processInstanceId);

Review Comment:
   Yes, I understand.
   usually data processing is required to be completed on the same day,
   but there are other tasks that need to be performed continuously every day and not sensitive to date delays.
   upstream data may have acceptable latency, when the delayed data is released,
   we usually expect the scheduling platform to automatically and continuously process the delayed data (including tasks that are blocked downstream for many days), especially when some third-party data access is involved.
   
   Some scheduling platforms have the feature of automatic date flop of tasks, especially this feature can also be implemented through the `dependent task` of Dolphinscheduler, `system.biz.date` parameter can make sure the task can get the correct date, dependent task(depend self yesterday) can make sure the task runs in date continuously.
   
   and most important is, this update changed some rules, i can never use `complement` command to rerun the delayed process which include `dependent task`(it never worked with complement), that's why i submited this pr.



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r965712336


##########
dolphinscheduler-dao/src/main/resources/sql/upgrade/2.0.7_schema/postgresql/dolphinscheduler_ddl.sql:
##########
@@ -0,0 +1,20 @@
+/*
+ * 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.
+*/
+
+drop index if exists task_instance_idx_code_pid_edtm;
+create index task_instance_idx_code_pid_edtm on t_ds_task_instance (task_code, process_instance_id, end_time);

Review Comment:
   it's a performance optimize if use this pr, on my platform, it have 5000+ dependent task to run, it generate 15000 records per day in t_ds_task_instance, in preview version, I added index (task_code, end_time)(2.0.6) and (process_instance_id, flag, start_time)(<=2.0.5), if don't create this index, a large number of full table scans will lead to huge resource consumption and the scheduler's database will be paralyzed.



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] JinyLeeChina commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
JinyLeeChina commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r965566764


##########
dolphinscheduler-dao/src/main/resources/sql/upgrade/2.0.7_schema/postgresql/dolphinscheduler_ddl.sql:
##########
@@ -0,0 +1,20 @@
+/*
+ * 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.
+*/
+
+drop index if exists task_instance_idx_code_pid_edtm;
+create index task_instance_idx_code_pid_edtm on t_ds_task_instance (task_code, process_instance_id, end_time);

Review Comment:
   I think adding this has nothing to do with this issue. Please pull request separately



##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java:
##########
@@ -81,7 +81,7 @@ IPage<TaskInstance> queryTaskInstanceListPaging(IPage<TaskInstance> page,
      * @param endTime endTime
      * @return task instance
      */
-    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("startTime") Date startTime, @Param("endTime") Date endTime);
+    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("processInstanceId") long processInstanceId);

Review Comment:
   When relying on specific tasks, I think that the execution of tasks in the time cycle has nothing to do with which workflow instance, so this issue cannot be modified in this way



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] JinyLeeChina closed pull request #11677: [Fix][2.0.6-2.0.7] Fix #11725 dependent task cannot detect target task finished across days

Posted by "JinyLeeChina (via GitHub)" <gi...@apache.org>.
JinyLeeChina closed pull request #11677: [Fix][2.0.6-2.0.7] Fix #11725 dependent task cannot detect target task finished across days
URL: https://github.com/apache/dolphinscheduler/pull/11677


-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r965682277


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java:
##########
@@ -81,7 +81,7 @@ IPage<TaskInstance> queryTaskInstanceListPaging(IPage<TaskInstance> page,
      * @param endTime endTime
      * @return task instance
      */
-    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("startTime") Date startTime, @Param("endTime") Date endTime);
+    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("processInstanceId") long processInstanceId);

Review Comment:
   because taskInstance don't keep the `scheduleTime`, but processInstance do, if only use startTime, and processInstance start with a scheduleTime, the dependent node will use `scheduleTime` to find the task, it may can't detect the task in original `scheduleTime`.
   
   the actual situation I have encountered is:
   etl task `B` processes a zipper table, it must run continuously by date, and used the parameter `system.biz.date`, it's previews dependent task `D`  depends on `A[T]` and `B[T-1](self yesterday)`, in some reason, `A` blocked until the `next day`, so `B.startTime` is `[T+1]`, and next day `D` will use `scheduleTime[T]` calculated by `scheduleTime[T+1]`  to finding `B[T]`, there is no `B` started at time `[T]`, so the dependent task `D[T+1]` will never finished, every time this happens, I need to manually stop the processInstance, and set the state of `D` to `Forced Success`, and recover the process.
   
   In fact taskInstance can only run attached to a processInstance, and taskInstance's system.biz.date parameter is same as processInstance, so I think it is right to find taskInstance through processInstance, and in 2.0.5, it did in the same way.



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r965682277


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java:
##########
@@ -81,7 +81,7 @@ IPage<TaskInstance> queryTaskInstanceListPaging(IPage<TaskInstance> page,
      * @param endTime endTime
      * @return task instance
      */
-    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("startTime") Date startTime, @Param("endTime") Date endTime);
+    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("processInstanceId") long processInstanceId);

Review Comment:
   because taskInstance don't keep the `scheduleTime`, but processInstance do, if only use start_time, and processInstance start with a scheduleTime, the dependent node will use `scheduleTime` to calculate a `bizdate` to find the task, it may can't detect the task in original `bizdate`.
   
   the actual situation I have encountered is:
   etl task `B` processes a zipper table, it must run continuously by date, it's previews dependent task `D`  depends on `A[T]` and `B[T-1](self yesterday)`, in some reason, `A` blocked until the `next day`, so `B.startTime` is `[T+1]`, and next day `D` will use `bizdate[T]` calculated by `scheduleTime[T+1]`  to finding `B[T]`, there is no `B` started on time `[T]`, that maks 
    dependent task `D[T+1]` will never stop, every time this happens, I need to manually stop the processInstance, and set the state of `D` to `Forced Success`.
   
   In fact taskInstance can only run attached to a processInstance, so I think it is right to find taskInstance through processInstance, and in 2.0.5, it did in the same way.



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r965712336


##########
dolphinscheduler-dao/src/main/resources/sql/upgrade/2.0.7_schema/postgresql/dolphinscheduler_ddl.sql:
##########
@@ -0,0 +1,20 @@
+/*
+ * 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.
+*/
+
+drop index if exists task_instance_idx_code_pid_edtm;
+create index task_instance_idx_code_pid_edtm on t_ds_task_instance (task_code, process_instance_id, end_time);

Review Comment:
   it's a performance optimize if use this pr, on my platform, it have 5000+ dependent task to run, and generated 15000 records every day in t_ds_task_instance, in preview version, I added index (task_code, end_time)(2.0.6) and (process_instance_id, flag, start_time)(<=2.0.5), if don't create this index, a large number of full table scans will lead to huge resource consumption and the scheduler's database will be paralyzed.



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r965682277


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java:
##########
@@ -81,7 +81,7 @@ IPage<TaskInstance> queryTaskInstanceListPaging(IPage<TaskInstance> page,
      * @param endTime endTime
      * @return task instance
      */
-    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("startTime") Date startTime, @Param("endTime") Date endTime);
+    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("processInstanceId") long processInstanceId);

Review Comment:
   because taskInstance don't keep the `scheduleTime`, but processInstance do, if only use startTime, and processInstance start with a scheduleTime, the dependent node will use `scheduleTime` to find the task, it may can't detect the task in original `scheduleTime`.
   
   the actual situation I have encountered is:
   etl task `B` processes a zipper table, it must run continuously by date, it's previews dependent task `D`  depends on `A[T]` and `B[T-1](self yesterday)`, in some reason, `A` blocked until the `next day`, so `B.startTime` is `[T+1]`, and next day `D` will use `bizdate[T]` calculated by `scheduleTime[T+1]`  to finding `B[T]`, there is no `B` started on time `[T]`, so the dependent task `D[T+1]` will never stop, every time this happens, I need to manually stop the processInstance, and set the state of `D` to `Forced Success`, and recover the process.
   
   In fact taskInstance can only run attached to a processInstance, and taskInstance's system.biz.date parameter is same as processInstance, so I think it is right to find taskInstance through processInstance, and in 2.0.5, it did in the same way.



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r966887972


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java:
##########
@@ -81,7 +81,7 @@ IPage<TaskInstance> queryTaskInstanceListPaging(IPage<TaskInstance> page,
      * @param endTime endTime
      * @return task instance
      */
-    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("startTime") Date startTime, @Param("endTime") Date endTime);
+    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("processInstanceId") long processInstanceId);

Review Comment:
   Yes, I understand.
   usually data processing is required to be completed on the same day,
   but there are other tasks that need to be performed continuously every day and not sensitive to date delays.
   upstream data may have acceptable latency, when the delayed data is released,
   at that time, we usually expect the scheduling platform to automatically and continuously process the delayed data (including tasks that are blocked downstream for many days), especially when some third-party data access is involved.
   
   Some scheduling platforms have the feature of automatic date flop of tasks, especially this feature can also be implemented through the `dependent task` of Dolphinscheduler.
   
   most important is, this update changed some rules, i can never use `complement` command to rerun the delayed process which include `dependent task`(it never worked with complement), that why i submited this pr.



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r965682277


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java:
##########
@@ -81,7 +81,7 @@ IPage<TaskInstance> queryTaskInstanceListPaging(IPage<TaskInstance> page,
      * @param endTime endTime
      * @return task instance
      */
-    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("startTime") Date startTime, @Param("endTime") Date endTime);
+    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("processInstanceId") long processInstanceId);

Review Comment:
   because taskInstance don't keep the `scheduleTime`, but processInstance do, if only use startTime, and processInstance start with a scheduleTime, the dependent node will use `scheduleTime` to calculate a `bizdate` to find the task, it may can't detect the task in original `bizdate`.
   
   the actual situation I have encountered is:
   etl task `B` processes a zipper table, it must run continuously by date, it's previews dependent task `D`  depends on `A[T]` and `B[T-1](self yesterday)`, in some reason, `A` blocked until the `next day`, so `B.startTime` is `[T+1]`, and next day `D` will use `bizdate[T]` calculated by `scheduleTime[T+1]`  to finding `B[T]`, there is no `B` started on time `[T]`, so the dependent task `D[T+1]` will never stop, every time this happens, I need to manually stop the processInstance, and set the state of `D` to `Forced Success`, and recover the process.
   
   In fact taskInstance can only run attached to a processInstance, so I think it is right to find taskInstance through processInstance, and in 2.0.5, it did in the same way.



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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


[GitHub] [dolphinscheduler] reele commented on a diff in pull request #11677: [Fix][2.0.6-2.0.7] Fix dependent task cannot detect target task finished across days

Posted by GitBox <gi...@apache.org>.
reele commented on code in PR #11677:
URL: https://github.com/apache/dolphinscheduler/pull/11677#discussion_r973815827


##########
dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskInstanceMapper.java:
##########
@@ -81,7 +81,7 @@ IPage<TaskInstance> queryTaskInstanceListPaging(IPage<TaskInstance> page,
      * @param endTime endTime
      * @return task instance
      */
-    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("startTime") Date startTime, @Param("endTime") Date endTime);
+    TaskInstance queryLastTaskInstance(@Param("taskCode") long taskCode, @Param("processInstanceId") long processInstanceId);

Review Comment:
   > A task should be finished yesterday
   
   This is just one of the cases, what if the task is allowed to finish in the next days?



-- 
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: commits-unsubscribe@dolphinscheduler.apache.org

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