Skip to content

Commit

Permalink
feat(autoware_pointcloud_preprocessor): redesign concatenate and time…
Browse files Browse the repository at this point in the history
… sync node (#8300)

* chore: rebase main

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: solve conflicts

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix cpp check

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: add diagnostics readme

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: update figure

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: upload jitter.png and add old design link

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: add the link to the tool for analyzing timestamp

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* fix: fix bug that timer didn't cancel

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix logic for logging

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* Update sensing/autoware_pointcloud_preprocessor/docs/concatenate-data.md

Co-authored-by: Max Schmeller <6088931+mojomex@users.noreply.github.com>

* Update sensing/autoware_pointcloud_preprocessor/src/concatenate_data/combine_cloud_handler.cpp

Co-authored-by: Max Schmeller <6088931+mojomex@users.noreply.github.com>

* Update sensing/autoware_pointcloud_preprocessor/schema/cocatenate_and_time_sync_node.schema.json

Co-authored-by: Max Schmeller <6088931+mojomex@users.noreply.github.com>

* Update sensing/autoware_pointcloud_preprocessor/schema/cocatenate_and_time_sync_node.schema.json

Co-authored-by: Max Schmeller <6088931+mojomex@users.noreply.github.com>

* Update sensing/autoware_pointcloud_preprocessor/src/concatenate_data/combine_cloud_handler.cpp

Co-authored-by: Max Schmeller <6088931+mojomex@users.noreply.github.com>

* Update sensing/autoware_pointcloud_preprocessor/src/concatenate_data/combine_cloud_handler.cpp

Co-authored-by: Max Schmeller <6088931+mojomex@users.noreply.github.com>

* chore: remove distortion corrector related changes

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* feat: add managed tf buffer

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix filename

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: add explanataion for maximum queue size

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: add explanation for timeout_sec

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix schema's explanation

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix description for twist and odom

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: remove license that are not used

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: change guard to prama once

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: default value change to string

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* Update sensing/autoware_pointcloud_preprocessor/test/test_concatenate_node_unit.cpp

Co-authored-by: Max Schmeller <6088931+mojomex@users.noreply.github.com>

* Update sensing/autoware_pointcloud_preprocessor/test/test_concatenate_node_unit.cpp

Co-authored-by: Max Schmeller <6088931+mojomex@users.noreply.github.com>

* Update sensing/autoware_pointcloud_preprocessor/test/test_concatenate_node_unit.cpp

Co-authored-by: Max Schmeller <6088931+mojomex@users.noreply.github.com>

* Update sensing/autoware_pointcloud_preprocessor/test/test_concatenate_node_unit.cpp

Co-authored-by: Max Schmeller <6088931+mojomex@users.noreply.github.com>

* style(pre-commit): autofix

* chore: clang-tidy style for static constexpr

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: remove unused vector header

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix naming concatenated_cloud_publisher

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix namimg diagnostic_updater_

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: specify parameter in comment

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: change RCLCPP_WARN to RCLCPP_WARN_STREAM_THROTTLE

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: add comment for cancelling timer

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: Simplify loop structure for topic-to-cloud mapping

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix spell errors

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix more spell error

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: rename mutex and lock

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: const reference for string parameter

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: add explaination for RclcppTimeHash_

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: change the concatenate node to parent node

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: clean processOdometry and processTwist

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: change twist shared pointer queue to twist queue

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: refactor compensate pointcloud to function

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: reallocate memory for concatenate_cloud_ptr

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: remove new to make shared

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: dis to distance

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: refacotr poitncloud_sub

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: return early return but throw runtime error

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: replace #define DEFAULT_SYNC_TOPIC_POSTFIX with member variable

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix spell error

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: remove redundant function call

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: replace conplex tuple to structure

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: use reference instead of a pointer to conveys node

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix camel to snake case

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix logic of publish synchronized pointcloud

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix cpp check

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: remove logging and throw error directly

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix clangd warnings

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix json schema

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix clangd warning

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: remove unused variable

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix launcher

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix clangd warning

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: ensure thread safety

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* style(pre-commit): autofix

* chore: clean code

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: add parameters for handling rosbag replay in loops

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix diagonistic

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: reduce copy operation

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: reserve space for concatenated pointcloud

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix clangd error

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix pipeline latency

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: add debug mode

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: refactor convert_to_xyzirc_cloud function

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix json schema

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix logging output

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix the output order of the debug mode

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix pipeline latency output

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: clean code

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: set some parameters to false in testing

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix default value for schema

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix diagnostic msgs

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix parameter for sample ros bag

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: update readme

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix empty pointcloud

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: remove duplicated logic

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix logic for handling empty pointcloud

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: clean code

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: remove rosbag_replay parameter

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: remove nodelet cpp

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: clang tidy warning

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* feat: add naive approach for unsynchronized pointclouds

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: add more explanations in docs for naive approach

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* feat: refactor naive method and fix the multithreading issue

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: set parameter to naive

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix parameter

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix readme

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* Update sensing/autoware_pointcloud_preprocessor/docs/concatenate-data.md

Co-authored-by: Max Schmeller <6088931+mojomex@users.noreply.github.com>

* Update sensing/autoware_pointcloud_preprocessor/docs/concatenate-data.md

Co-authored-by: Max Schmeller <6088931+mojomex@users.noreply.github.com>

* style(pre-commit): autofix

* feat: remove mutually exclusive approaches

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix spell error

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: remove unused variable

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* refactor: refactor collectorInfo to polymorphic

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix variable name

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: fix figure and diagnostic msg in readme

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chroe: refactor collectorinfo structure

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: revert wrong file changes

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: improve message

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: remove unused input topics

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: change to explicit check

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: tier4 debug msgs to autoware internal debug msgs

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

* chore: update documentation

Signed-off-by: vividf <yihsiang.fang@tier4.jp>

---------

Signed-off-by: vividf <yihsiang.fang@tier4.jp>
Co-authored-by: Max Schmeller <6088931+mojomex@users.noreply.github.com>
Co-authored-by: pre-commit-ci[bot] <66853113+pre-commit-ci[bot]@users.noreply.github.com>
  • Loading branch information
3 people authored Jan 21, 2025
1 parent 3138d75 commit 96708f9
Show file tree
Hide file tree
Showing 27 changed files with 9,303 additions and 1,258 deletions.
16 changes: 14 additions & 2 deletions sensing/autoware_pointcloud_preprocessor/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,10 @@ ament_target_dependencies(faster_voxel_grid_downsample_filter
)

ament_auto_add_library(pointcloud_preprocessor_filter SHARED
src/concatenate_data/concatenate_and_time_sync_nodelet.cpp
src/concatenate_data/concatenate_and_time_sync_node.cpp
src/concatenate_data/combine_cloud_handler.cpp
src/concatenate_data/cloud_collector.cpp
src/concatenate_data/collector_matching_strategy.cpp
src/concatenate_data/concatenate_pointclouds.cpp
src/crop_box_filter/crop_box_filter_node.cpp
src/time_synchronizer/time_synchronizer_node.cpp
Expand Down Expand Up @@ -111,7 +114,7 @@ rclcpp_components_register_node(pointcloud_preprocessor_filter
# ========== Concatenate and Sync data ==========
rclcpp_components_register_node(pointcloud_preprocessor_filter
PLUGIN "autoware::pointcloud_preprocessor::PointCloudConcatenateDataSynchronizerComponent"
EXECUTABLE concatenate_data_node)
EXECUTABLE concatenate_and_time_sync_node)

# ========== CropBox ==========
rclcpp_components_register_node(pointcloud_preprocessor_filter
Expand Down Expand Up @@ -243,8 +246,17 @@ if(BUILD_TESTING)
test/test_distortion_corrector_node.cpp
)

ament_add_gtest(test_concatenate_node_unit
test/test_concatenate_node_unit.cpp
)

target_link_libraries(test_utilities pointcloud_preprocessor_filter)
target_link_libraries(test_distortion_corrector_node pointcloud_preprocessor_filter)
target_link_libraries(test_concatenate_node_unit pointcloud_preprocessor_filter)

add_ros_test(
test/test_concatenate_node_component.py
TIMEOUT "50"
)

endif()
Original file line number Diff line number Diff line change
@@ -0,0 +1,21 @@
/**:
ros__parameters:
debug_mode: false
has_static_tf_only: false
rosbag_length: 10.0
maximum_queue_size: 5
timeout_sec: 0.2
is_motion_compensated: true
publish_synchronized_pointcloud: true
keep_input_frame_in_synchronized_pointcloud: true
publish_previous_but_late_pointcloud: false
synchronized_pointcloud_postfix: pointcloud
input_twist_topic_type: twist
input_topics: [
"/sensing/lidar/right/pointcloud_before_sync",
"/sensing/lidar/top/pointcloud_before_sync",
"/sensing/lidar/left/pointcloud_before_sync",
]
output_frame: base_link
matching_strategy:
type: naive
235 changes: 192 additions & 43 deletions sensing/autoware_pointcloud_preprocessor/docs/concatenate-data.md

Large diffs are not rendered by default.

Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.
Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.

This file was deleted.

Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.
Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.
Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.
Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.
Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.
Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.
Loading
Sorry, something went wrong. Reload?
Sorry, we cannot display this file.
Sorry, this file is invalid so it cannot be displayed.
Original file line number Diff line number Diff line change
@@ -0,0 +1,89 @@
// Copyright 2024 TIER IV, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

#pragma once

#include "combine_cloud_handler.hpp"

#include <memory>
#include <string>
#include <unordered_map>

namespace autoware::pointcloud_preprocessor
{

class PointCloudConcatenateDataSynchronizerComponent;
class CombineCloudHandler;

struct CollectorInfoBase
{
virtual ~CollectorInfoBase() = default;
};

struct NaiveCollectorInfo : public CollectorInfoBase
{
double timestamp;

explicit NaiveCollectorInfo(double timestamp = 0.0) : timestamp(timestamp) {}
};

struct AdvancedCollectorInfo : public CollectorInfoBase
{
double timestamp;
double noise_window;

explicit AdvancedCollectorInfo(double timestamp = 0.0, double noise_window = 0.0)
: timestamp(timestamp), noise_window(noise_window)
{
}
};

class CloudCollector
{
public:
CloudCollector(
std::shared_ptr<PointCloudConcatenateDataSynchronizerComponent> && ros2_parent_node,
std::shared_ptr<CombineCloudHandler> & combine_cloud_handler, int num_of_clouds,
double timeout_sec, bool debug_mode);
bool topic_exists(const std::string & topic_name);
bool process_pointcloud(
const std::string & topic_name, sensor_msgs::msg::PointCloud2::SharedPtr cloud);
void concatenate_callback();

ConcatenatedCloudResult concatenate_pointclouds(
std::unordered_map<std::string, sensor_msgs::msg::PointCloud2::SharedPtr> topic_to_cloud_map);

std::unordered_map<std::string, sensor_msgs::msg::PointCloud2::SharedPtr>
get_topic_to_cloud_map();

[[nodiscard]] bool concatenate_finished() const;

void set_info(std::shared_ptr<CollectorInfoBase> collector_info);
[[nodiscard]] std::shared_ptr<CollectorInfoBase> get_info() const;
void show_debug_message();

private:
std::shared_ptr<PointCloudConcatenateDataSynchronizerComponent> ros2_parent_node_;
std::shared_ptr<CombineCloudHandler> combine_cloud_handler_;
rclcpp::TimerBase::SharedPtr timer_;
std::unordered_map<std::string, sensor_msgs::msg::PointCloud2::SharedPtr> topic_to_cloud_map_;
uint64_t num_of_clouds_;
double timeout_sec_;
bool debug_mode_;
bool concatenate_finished_{false};
std::mutex concatenate_mutex_;
std::shared_ptr<CollectorInfoBase> collector_info_;
};

} // namespace autoware::pointcloud_preprocessor
Original file line number Diff line number Diff line change
@@ -0,0 +1,81 @@
// Copyright 2024 TIER IV, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

#pragma once

#include "cloud_collector.hpp"

#include <rclcpp/node.hpp>

#include <sensor_msgs/msg/point_cloud2.hpp>

#include <list>
#include <memory>
#include <optional>
#include <string>
#include <unordered_map>
#include <vector>

namespace autoware::pointcloud_preprocessor
{

struct MatchingParams
{
std::string topic_name;
double cloud_timestamp;
double cloud_arrival_time;
};

class CollectorMatchingStrategy
{
public:
virtual ~CollectorMatchingStrategy() = default;

[[nodiscard]] virtual std::optional<std::shared_ptr<CloudCollector>> match_cloud_to_collector(
const std::list<std::shared_ptr<CloudCollector>> & cloud_collectors,
const MatchingParams & params) const = 0;
virtual void set_collector_info(
std::shared_ptr<CloudCollector> & collector, const MatchingParams & matching_params) = 0;
};

class NaiveMatchingStrategy : public CollectorMatchingStrategy
{
public:
explicit NaiveMatchingStrategy(rclcpp::Node & node);
[[nodiscard]] std::optional<std::shared_ptr<CloudCollector>> match_cloud_to_collector(
const std::list<std::shared_ptr<CloudCollector>> & cloud_collectors,
const MatchingParams & params) const override;
void set_collector_info(
std::shared_ptr<CloudCollector> & collector, const MatchingParams & matching_params) override;
};

class AdvancedMatchingStrategy : public CollectorMatchingStrategy
{
public:
explicit AdvancedMatchingStrategy(rclcpp::Node & node, std::vector<std::string> input_topics);

[[nodiscard]] std::optional<std::shared_ptr<CloudCollector>> match_cloud_to_collector(
const std::list<std::shared_ptr<CloudCollector>> & cloud_collectors,
const MatchingParams & params) const override;
void set_collector_info(
std::shared_ptr<CloudCollector> & collector, const MatchingParams & matching_params) override;

private:
std::unordered_map<std::string, double> topic_to_offset_map_;
std::unordered_map<std::string, double> topic_to_noise_window_map_;
};

std::shared_ptr<CollectorMatchingStrategy> parse_matching_strategy(rclcpp::Node & node);

} // namespace autoware::pointcloud_preprocessor
Original file line number Diff line number Diff line change
@@ -0,0 +1,110 @@
// Copyright 2024 TIER IV, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

#pragma once

#include <deque>
#include <memory>
#include <optional>
#include <string>
#include <unordered_map>
#include <vector>

// ROS includes
#include "autoware/point_types/types.hpp"

#include <autoware/universe_utils/ros/managed_transform_buffer.hpp>
#include <diagnostic_updater/diagnostic_updater.hpp>
#include <point_cloud_msg_wrapper/point_cloud_msg_wrapper.hpp>

#include <diagnostic_msgs/msg/diagnostic_status.hpp>
#include <geometry_msgs/msg/twist_stamped.hpp>
#include <geometry_msgs/msg/twist_with_covariance_stamped.hpp>
#include <nav_msgs/msg/odometry.hpp>
#include <sensor_msgs/msg/point_cloud2.hpp>
#include <sensor_msgs/point_cloud2_iterator.hpp>
#include <tier4_debug_msgs/msg/int32_stamped.hpp>
#include <tier4_debug_msgs/msg/string_stamped.hpp>

#include <message_filters/pass_through.h>
#include <message_filters/subscriber.h>
#include <message_filters/sync_policies/approximate_time.h>
#include <message_filters/sync_policies/exact_time.h>
#include <message_filters/synchronizer.h>

namespace autoware::pointcloud_preprocessor
{
using autoware::point_types::PointXYZIRC;
using point_cloud_msg_wrapper::PointCloud2Modifier;

struct ConcatenatedCloudResult
{
sensor_msgs::msg::PointCloud2::SharedPtr concatenate_cloud_ptr{nullptr};
std::optional<std::unordered_map<std::string, sensor_msgs::msg::PointCloud2::SharedPtr>>
topic_to_transformed_cloud_map;
std::unordered_map<std::string, double> topic_to_original_stamp_map;
};

class CombineCloudHandler
{
private:
rclcpp::Node & node_;

std::string output_frame_;
bool is_motion_compensated_;
bool publish_synchronized_pointcloud_;
bool keep_input_frame_in_synchronized_pointcloud_;
std::unique_ptr<autoware::universe_utils::ManagedTransformBuffer> managed_tf_buffer_{nullptr};

std::deque<geometry_msgs::msg::TwistStamped> twist_queue_;

/// @brief RclcppTimeHash structure defines a custom hash function for the rclcpp::Time type by
/// using its nanoseconds representation as the hash value.
struct RclcppTimeHash
{
std::size_t operator()(const rclcpp::Time & t) const
{
return std::hash<int64_t>()(t.nanoseconds());
}
};

static void convert_to_xyzirc_cloud(
const sensor_msgs::msg::PointCloud2::SharedPtr & input_cloud,
sensor_msgs::msg::PointCloud2::SharedPtr & xyzirc_cloud);

void correct_pointcloud_motion(
const std::shared_ptr<sensor_msgs::msg::PointCloud2> & transformed_cloud_ptr,
const std::vector<rclcpp::Time> & pc_stamps,
std::unordered_map<rclcpp::Time, Eigen::Matrix4f, RclcppTimeHash> & transform_memo,
std::shared_ptr<sensor_msgs::msg::PointCloud2> transformed_delay_compensated_cloud_ptr);

public:
CombineCloudHandler(
rclcpp::Node & node, std::string output_frame, bool is_motion_compensated,
bool publish_synchronized_pointcloud, bool keep_input_frame_in_synchronized_pointcloud,
bool has_static_tf_only);
void process_twist(
const geometry_msgs::msg::TwistWithCovarianceStamped::ConstSharedPtr & twist_msg);
void process_odometry(const nav_msgs::msg::Odometry::ConstSharedPtr & odometry_msg);

ConcatenatedCloudResult combine_pointclouds(
std::unordered_map<std::string, sensor_msgs::msg::PointCloud2::SharedPtr> & topic_to_cloud_map);

Eigen::Matrix4f compute_transform_to_adjust_for_old_timestamp(
const rclcpp::Time & old_stamp, const rclcpp::Time & new_stamp);

std::deque<geometry_msgs::msg::TwistStamped> get_twist_queue();
};

} // namespace autoware::pointcloud_preprocessor
Loading

0 comments on commit 96708f9

Please sign in to comment.