Merge "Make image viewer polling rate configurable."
diff --git a/WORKSPACE b/WORKSPACE
index 3e1801c..aa92b7e 100644
--- a/WORKSPACE
+++ b/WORKSPACE
@@ -549,6 +549,18 @@
actual = "@com_google_googletest//:gtest_main",
)
+http_archive(
+ name = "april_tag_test_image",
+ build_file_content = """
+filegroup(
+ name = "april_tag_test_image",
+ srcs = ["test.bfbs", "expected.jpeg", "expected.png"],
+ visibility = ["//visibility:public"],
+)""",
+ sha256 = "5312c79b19e9883b3cebd9d65b4438a2bf05b41da0bcd8c35e19d22c3b2e1859",
+ urls = ["https://www.frc971.org/Build-Dependencies/test_image_frc971.vision.CameraImage_2023.01.28.tar.gz"],
+)
+
# Recompressed from libusb-1.0.21.7z.
http_file(
name = "libusb_1_0_windows",
diff --git a/aos/util/BUILD b/aos/util/BUILD
index 48dc054..929b376 100644
--- a/aos/util/BUILD
+++ b/aos/util/BUILD
@@ -466,6 +466,28 @@
)
cc_library(
+ name = "threaded_consumer",
+ hdrs = [
+ "threaded_consumer.h",
+ ],
+ deps = [
+ "//aos:condition",
+ "//aos:realtime",
+ "//aos/containers:ring_buffer",
+ "//aos/mutex",
+ ],
+)
+
+cc_test(
+ name = "threaded_consumer_test",
+ srcs = ["threaded_consumer_test.cc"],
+ deps = [
+ ":threaded_consumer",
+ "//aos/testing:googletest",
+ ],
+)
+
+cc_library(
name = "foxglove_websocket_lib",
srcs = ["foxglove_websocket_lib.cc"],
hdrs = ["foxglove_websocket_lib.h"],
diff --git a/aos/util/threaded_consumer.h b/aos/util/threaded_consumer.h
new file mode 100644
index 0000000..95ec79a
--- /dev/null
+++ b/aos/util/threaded_consumer.h
@@ -0,0 +1,102 @@
+#ifndef AOS_UTIL_THREADED_CONSUMER_H_
+#define AOS_UTIL_THREADED_CONSUMER_H_
+
+#include <functional>
+#include <optional>
+#include <thread>
+
+#include "aos/condition.h"
+#include "aos/containers/ring_buffer.h"
+#include "aos/mutex/mutex.h"
+#include "aos/realtime.h"
+
+namespace aos {
+namespace util {
+
+// This class implements a threadpool of a single worker that accepts work
+// from the main thread through a queue and executes it at a different realtime
+// priority.
+//
+// There is no mechanism to get data back to the main thread, the worker only
+// acts as a consumer. When this class is destroyed, it join()s the worker and
+// finishes all outstanding tasks.
+template <typename T, int QueueSize>
+class ThreadedConsumer {
+ public:
+ // Constructs a new ThreadedConsumer with the given consumer function to be
+ // run at the given realtime priority. If worker_priority is zero, the thread
+ // will stay at non realtime priority.
+ ThreadedConsumer(std::function<void(T)> consumer_function,
+ int worker_priority)
+ : consumer_function_(consumer_function),
+ worker_priority_(worker_priority),
+ more_tasks_(&mutex_),
+ worker_thread_([this]() { WorkerFunction(); }) {}
+
+ ~ThreadedConsumer() {
+ {
+ aos::MutexLocker locker(&mutex_);
+ quit_ = true;
+ more_tasks_.Broadcast();
+ }
+ worker_thread_.join();
+ }
+
+ // Submits another task to be processed by the worker.
+ // Returns true if successfully pushed onto the queue, and false if the queue
+ // is full.
+ bool Push(T task) {
+ aos::MutexLocker locker(&mutex_);
+
+ if (task_queue_.full()) {
+ return false;
+ }
+
+ task_queue_.Push(task);
+ more_tasks_.Broadcast();
+
+ return true;
+ }
+
+ private:
+ void WorkerFunction() {
+ if (worker_priority_ > 0) {
+ aos::SetCurrentThreadRealtimePriority(worker_priority_);
+ }
+
+ while (true) {
+ std::optional<T> task;
+
+ {
+ aos::MutexLocker locker(&mutex_);
+ while (task_queue_.empty() && !quit_) {
+ CHECK(!more_tasks_.Wait());
+ }
+
+ if (task_queue_.empty() && quit_) break;
+
+ // Pop
+ task = std::move(task_queue_[0]);
+ task_queue_.Shift();
+ }
+
+ consumer_function_(*task);
+ task.reset();
+ }
+
+ aos::UnsetCurrentThreadRealtimePriority();
+ }
+
+ std::function<void(T)> consumer_function_;
+ aos::RingBuffer<T, QueueSize> task_queue_;
+ aos::Mutex mutex_;
+ bool quit_ = false;
+ int worker_priority_;
+ aos::Condition more_tasks_;
+ std::thread worker_thread_;
+};
+
+} // namespace util
+} // namespace aos
+
+#endif // AOS_UTIL_THREADWORKER_H_
diff --git a/aos/util/threaded_consumer_test.cc b/aos/util/threaded_consumer_test.cc
new file mode 100644
index 0000000..f137108
--- /dev/null
+++ b/aos/util/threaded_consumer_test.cc
@@ -0,0 +1,144 @@
+#include "aos/util/threaded_consumer.h"
+
+#include "gtest/gtest.h"
+
+namespace aos {
+namespace util {
+
+// We expect it to be able to pass through everything we submit and recieves it
+// in the order that we submitted it. It should also be able to take in more
+// tasks than the size of the ring buffer as long as the worker doesn't get
+// behind.
+TEST(ThreadedConsumerTest, BasicFunction) {
+ std::atomic<int> counter{0};
+
+ ThreadedConsumer<int, 4> threaded_consumer(
+ [&counter](int task) {
+ LOG(INFO) << "task:" << task << " counter: " << counter;
+ counter = task;
+ },
+ 0);
+
+ for (int number : {9, 7, 1, 3, 100, 300, 42}) {
+ EXPECT_TRUE(threaded_consumer.Push(number));
+
+ // wait
+ while (counter != number) {
+ std::this_thread::sleep_for(std::chrono::milliseconds(1));
+ }
+
+ EXPECT_EQ(counter, number);
+ }
+}
+
+// We should be able to raise the realtime priority of the worker thread, and
+// everything should work the same. It should also reset back to lower priority
+// when shutting down the worker thread.
+TEST(ThreadedConsumerTest, ElevatedPriority) {
+ std::atomic<int> counter{0};
+
+ {
+ ThreadedConsumer<int, 4> threaded_consumer(
+ [&counter](int task) {
+ CheckRealtime();
+ LOG(INFO) << "task:" << task << " counter: " << counter;
+ counter = task;
+ },
+ 20);
+
+ for (int number : {9, 7, 1, 3, 100, 300, 42}) {
+ EXPECT_TRUE(threaded_consumer.Push(number));
+
+ // wait
+ while (counter != number) {
+ std::this_thread::sleep_for(std::chrono::milliseconds(1));
+ }
+
+ EXPECT_EQ(counter, number);
+ }
+ }
+ // TODO: Check that the worker thread's priority actually gets reset before
+ // the thread is destroyed.
+
+ CheckNotRealtime();
+}
+
+// If the worker gets behind, we shouldn't silently take in more tasks and
+// destroy old ones.
+TEST(ThreadedConsumerTest, OverflowRingBuffer) {
+ std::atomic<int> counter{0};
+ std::atomic<int> should_block{true};
+
+ ThreadedConsumer<int, 4> threaded_consumer(
+ [&counter, &should_block](int task) {
+ LOG(INFO) << "task:" << task << " counter: " << counter;
+
+ counter = task;
+
+ // prevent it from making any progress to simulate it getting behind
+ while (should_block) {
+ std::this_thread::sleep_for(std::chrono::milliseconds(1));
+ }
+ },
+ 20);
+
+ // It consumes the 5 and then our worker blocks.
+ EXPECT_TRUE(threaded_consumer.Push(5));
+
+ // Wait for it to consume 5
+ while (counter != 5) {
+ std::this_thread::sleep_for(std::chrono::milliseconds(1));
+ };
+
+ // 4 more fills up the queue.
+ for (int number : {8, 9, 7, 1}) {
+ EXPECT_TRUE(threaded_consumer.Push(number));
+ }
+
+ // this one should overflow the buffer.
+ EXPECT_FALSE(threaded_consumer.Push(101));
+
+ // clean up, so we don't join() an infinite loop
+ should_block = false;
+}
+
+// The class should destruct gracefully and finish all of its work before
+// dissapearing.
+TEST(ThreadedConsumerTest, FinishesTasksOnQuit) {
+ std::atomic<int> counter{0};
+ std::atomic<int> should_block{true};
+
+ {
+ ThreadedConsumer<int, 4> threaded_consumer(
+ [&counter, &should_block](int task) {
+ LOG(INFO) << "task:" << task << " counter: " << counter;
+
+ counter = task;
+
+ // prevent it from making any progress to simulate it getting behind
+ while (should_block) {
+ std::this_thread::sleep_for(std::chrono::milliseconds(1));
+ }
+ },
+ 20);
+
+ // Give it some work to do
+ for (int number : {8, 9, 7, 1}) {
+ EXPECT_TRUE(threaded_consumer.Push(number));
+ }
+
+ // Wait for it to consume the first number
+ while (counter != 8) {
+ std::this_thread::sleep_for(std::chrono::milliseconds(1));
+ };
+
+ // allow it to continue
+ should_block = false;
+ }
+
+ // It should have finished all the work and gotten to the last number.
+ EXPECT_EQ(counter, 1);
+}
+
+} // namespace util
+} // namespace aos
diff --git a/frc971/control_loops/python/basic_window.py b/frc971/control_loops/python/basic_window.py
index 44e4f49..886a5d3 100755
--- a/frc971/control_loops/python/basic_window.py
+++ b/frc971/control_loops/python/basic_window.py
@@ -7,7 +7,7 @@
from gi.repository import Gdk
from gi.repository import GdkX11
import cairo
-from constants import *
+from frc971.control_loops.python.constants import *
identity = cairo.Matrix()
diff --git a/frc971/vision/BUILD b/frc971/vision/BUILD
index c52afae..642ca0f 100644
--- a/frc971/vision/BUILD
+++ b/frc971/vision/BUILD
@@ -1,4 +1,5 @@
load("@com_github_google_flatbuffers//:build_defs.bzl", "flatbuffer_cc_library", "flatbuffer_py_library")
+load("//aos:config.bzl", "aos_config")
load("@com_github_google_flatbuffers//:typescript.bzl", "flatbuffer_ts_library")
flatbuffer_cc_library(
@@ -76,6 +77,7 @@
"//aos/events:epoll",
"//aos/events:event_loop",
"//aos/scoped:scoped_fd",
+ "//aos/util:threaded_consumer",
"@com_github_google_glog//:glog",
"@com_google_absl//absl/base",
],
@@ -102,6 +104,7 @@
"//y2020/vision/sift:sift_fbs",
"//y2020/vision/sift:sift_training_fbs",
"//y2020/vision/tools/python_code:sift_training_data",
+ "@com_github_foxglove_schemas//:schemas",
"@com_github_google_glog//:glog",
"@com_google_absl//absl/strings:str_format",
"@com_google_absl//absl/types:span",
@@ -121,6 +124,7 @@
visibility = ["//visibility:public"],
deps = [
":charuco_lib",
+ ":foxglove_image_converter",
"//aos:init",
"//aos/events/logging:log_reader",
"//frc971/analysis:in_process_plotter",
@@ -129,6 +133,8 @@
"//frc971/wpilib:imu_batch_fbs",
"//frc971/wpilib:imu_fbs",
"//third_party:opencv",
+ "@com_github_foxglove_schemas//:CompressedImage_schema",
+ "@com_github_foxglove_schemas//:ImageAnnotations_schema",
"@com_google_absl//absl/strings:str_format",
"@com_google_ceres_solver//:ceres",
"@org_tuxfamily_eigen//:eigen",
@@ -258,9 +264,42 @@
hdrs = ["foxglove_image_converter.h"],
visibility = ["//visibility:public"],
deps = [
+ ":charuco_lib",
":vision_fbs",
"//aos/events:event_loop",
"//third_party:opencv",
"@com_github_foxglove_schemas//:schemas",
],
)
+
+aos_config(
+ name = "converter_config",
+ testonly = True,
+ src = "converter_test_config.json",
+ flatbuffers = [
+ "//frc971/vision:vision_fbs",
+ "//aos/events:event_loop_fbs",
+ "//aos/logging:log_message_fbs",
+ "//aos/network:message_bridge_client_fbs",
+ "//aos/network:message_bridge_server_fbs",
+ "//aos/network:timestamp_fbs",
+ "@com_github_foxglove_schemas//:schemas",
+ ],
+)
+
+cc_test(
+ name = "foxglove_image_converter_test",
+ srcs = ["foxglove_image_converter_test.cc"],
+ data = [
+ ":converter_config",
+ "@april_tag_test_image",
+ ],
+ deps = [
+ ":foxglove_image_converter",
+ "//aos:configuration",
+ "//aos/events:simulated_event_loop",
+ "//aos/testing:googletest",
+ "//aos/testing:path",
+ "//aos/testing:tmpdir",
+ ],
+)
diff --git a/frc971/vision/calibration_accumulator.cc b/frc971/vision/calibration_accumulator.cc
index eee22f5..ea9af28 100644
--- a/frc971/vision/calibration_accumulator.cc
+++ b/frc971/vision/calibration_accumulator.cc
@@ -11,6 +11,8 @@
#include "frc971/control_loops/quaternion_utils.h"
#include "frc971/vision/charuco_lib.h"
#include "frc971/wpilib/imu_batch_generated.h"
+#include "external/com_github_foxglove_schemas/ImageAnnotations_schema.h"
+#include "external/com_github_foxglove_schemas/CompressedImage_schema.h"
DEFINE_bool(display_undistorted, false,
"If true, display the undistorted image.");
@@ -111,6 +113,33 @@
}
}
+CalibrationFoxgloveVisualizer::CalibrationFoxgloveVisualizer(
+ aos::EventLoop *event_loop)
+ : event_loop_(event_loop),
+ image_converter_(event_loop_, "/camera", "/visualization",
+ ImageCompression::kJpeg),
+ annotations_sender_(
+ event_loop_->MakeSender<foxglove::ImageAnnotations>("/visualization")) {}
+
+aos::FlatbufferDetachedBuffer<aos::Configuration>
+CalibrationFoxgloveVisualizer::AddVisualizationChannels(
+ const aos::Configuration *config, const aos::Node *node) {
+ constexpr std::string_view channel_name = "/visualization";
+ aos::ChannelT channel_overrides;
+ channel_overrides.max_size = 10000000;
+ aos::FlatbufferDetachedBuffer<aos::Configuration> result =
+ aos::configuration::AddChannelToConfiguration(
+ config, channel_name,
+ aos::FlatbufferSpan<reflection::Schema>(
+ foxglove::ImageAnnotationsSchema()),
+ node, channel_overrides);
+ return aos::configuration::AddChannelToConfiguration(
+ &result.message(), channel_name,
+ aos::FlatbufferSpan<reflection::Schema>(
+ foxglove::CompressedImageSchema()),
+ node, channel_overrides);
+}
+
Calibration::Calibration(aos::SimulatedEventLoopFactory *event_loop_factory,
aos::EventLoop *image_event_loop,
aos::EventLoop *imu_event_loop, std::string_view pi,
@@ -140,7 +169,9 @@
[this](cv::Mat rgb_image, const monotonic_clock::time_point eof) {
charuco_extractor_.HandleImage(rgb_image, eof);
}),
- data_(data) {
+ data_(data),
+ visualizer_event_loop_(image_factory_->MakeEventLoop("visualization")),
+ visualizer_(visualizer_event_loop_.get()) {
imu_factory_->OnShutdown([]() { cv::destroyAllWindows(); });
// Check for IMUValuesBatch topic on both /localizer and /drivetrain channels,
@@ -173,9 +204,10 @@
void Calibration::HandleCharuco(
cv::Mat rgb_image, const monotonic_clock::time_point eof,
std::vector<cv::Vec4i> /*charuco_ids*/,
- std::vector<std::vector<cv::Point2f>> /*charuco_corners*/, bool valid,
+ std::vector<std::vector<cv::Point2f>> charuco_corners, bool valid,
std::vector<Eigen::Vector3d> rvecs_eigen,
std::vector<Eigen::Vector3d> tvecs_eigen) {
+ visualizer_.HandleCharuco(eof, charuco_corners);
if (valid) {
CHECK(rvecs_eigen.size() > 0) << "Require at least one target detected";
// We only use one (the first) target detected for calibration
@@ -237,6 +269,7 @@
last_value_.accelerometer_y,
last_value_.accelerometer_z);
+ // TODO: ToDistributedClock may be too noisy.
data_->AddImu(imu_factory_->ToDistributedClock(monotonic_clock::time_point(
chrono::nanoseconds(imu->monotonic_timestamp_ns()))),
gyro, accel * kG);
diff --git a/frc971/vision/calibration_accumulator.h b/frc971/vision/calibration_accumulator.h
index d9f6065..5c435ad 100644
--- a/frc971/vision/calibration_accumulator.h
+++ b/frc971/vision/calibration_accumulator.h
@@ -8,6 +8,7 @@
#include "aos/time/time.h"
#include "frc971/control_loops/quaternion_utils.h"
#include "frc971/vision/charuco_lib.h"
+#include "frc971/vision/foxglove_image_converter.h"
#include "frc971/wpilib/imu_batch_generated.h"
namespace frc971 {
@@ -76,6 +77,28 @@
turret_points_;
};
+class CalibrationFoxgloveVisualizer {
+ public:
+ CalibrationFoxgloveVisualizer(aos::EventLoop *event_loop);
+
+ static aos::FlatbufferDetachedBuffer<aos::Configuration>
+ AddVisualizationChannels(const aos::Configuration *config,
+ const aos::Node *node);
+
+ void HandleCharuco(const aos::monotonic_clock::time_point eof,
+ std::vector<std::vector<cv::Point2f>> charuco_corners) {
+ auto builder = annotations_sender_.MakeBuilder();
+ builder.CheckOk(
+ builder.Send(BuildAnnotations(eof, charuco_corners, builder.fbb())));
+ }
+
+ private:
+ aos::EventLoop *event_loop_;
+ FoxgloveImageConverter image_converter_;
+
+ aos::Sender<foxglove::ImageAnnotations> annotations_sender_;
+};
+
// Class to register image and IMU callbacks in AOS and route them to the
// corresponding CalibrationData class.
class Calibration {
@@ -110,6 +133,9 @@
CalibrationData *data_;
+ std::unique_ptr<aos::EventLoop> visualizer_event_loop_;
+ CalibrationFoxgloveVisualizer visualizer_;
+
frc971::IMUValuesT last_value_;
};
diff --git a/frc971/vision/charuco_lib.cc b/frc971/vision/charuco_lib.cc
index 622c871..f12a6a5 100644
--- a/frc971/vision/charuco_lib.cc
+++ b/frc971/vision/charuco_lib.cc
@@ -480,5 +480,42 @@
rvecs_eigen, tvecs_eigen);
}
+flatbuffers::Offset<foxglove::ImageAnnotations> BuildAnnotations(
+ const aos::monotonic_clock::time_point monotonic_now,
+ const std::vector<std::vector<cv::Point2f>> &corners,
+ flatbuffers::FlatBufferBuilder *fbb) {
+ std::vector<flatbuffers::Offset<foxglove::PointsAnnotation>> rectangles;
+ const struct timespec now_t = aos::time::to_timespec(monotonic_now);
+ foxglove::Time time{static_cast<uint32_t>(now_t.tv_sec),
+ static_cast<uint32_t>(now_t.tv_nsec)};
+ const flatbuffers::Offset<foxglove::Color> color_offset =
+ foxglove::CreateColor(*fbb, 0.0, 1.0, 0.0, 1.0);
+ for (const std::vector<cv::Point2f> &rectangle : corners) {
+ std::vector<flatbuffers::Offset<foxglove::Point2>> points_offsets;
+ for (const cv::Point2f &point : rectangle) {
+ points_offsets.push_back(foxglove::CreatePoint2(*fbb, point.x, point.y));
+ }
+ const flatbuffers::Offset<
+ flatbuffers::Vector<flatbuffers::Offset<foxglove::Point2>>>
+ points_offset = fbb->CreateVector(points_offsets);
+ std::vector<flatbuffers::Offset<foxglove::Color>> color_offsets(
+ points_offsets.size(), color_offset);
+ auto colors_offset = fbb->CreateVector(color_offsets);
+ foxglove::PointsAnnotation::Builder points_builder(*fbb);
+ points_builder.add_timestamp(&time);
+ points_builder.add_type(foxglove::PointsAnnotationType::POINTS);
+ points_builder.add_points(points_offset);
+ points_builder.add_outline_color(color_offset);
+ points_builder.add_outline_colors(colors_offset);
+ points_builder.add_thickness(2.0);
+ rectangles.push_back(points_builder.Finish());
+ }
+
+ const auto rectangles_offset = fbb->CreateVector(rectangles);
+ foxglove::ImageAnnotations::Builder annotation_builder(*fbb);
+ annotation_builder.add_points(rectangles_offset);
+ return annotation_builder.Finish();
+}
+
} // namespace vision
} // namespace frc971
diff --git a/frc971/vision/charuco_lib.h b/frc971/vision/charuco_lib.h
index 7759482..984cef6 100644
--- a/frc971/vision/charuco_lib.h
+++ b/frc971/vision/charuco_lib.h
@@ -13,6 +13,7 @@
#include "aos/network/message_bridge_server_generated.h"
#include "y2020/vision/sift/sift_generated.h"
#include "y2020/vision/sift/sift_training_generated.h"
+#include "external/com_github_foxglove_schemas/ImageAnnotations_generated.h"
DECLARE_bool(visualize);
@@ -178,6 +179,13 @@
handle_charuco_;
};
+// Puts the provided charuco corners into a foxglove ImageAnnotation type for
+// visualization purposes.
+flatbuffers::Offset<foxglove::ImageAnnotations> BuildAnnotations(
+ const aos::monotonic_clock::time_point monotonic_now,
+ const std::vector<std::vector<cv::Point2f>> &corners,
+ flatbuffers::FlatBufferBuilder *fbb);
+
} // namespace vision
} // namespace frc971
diff --git a/frc971/vision/converter_test_config.json b/frc971/vision/converter_test_config.json
new file mode 100644
index 0000000..5d74dd1
--- /dev/null
+++ b/frc971/vision/converter_test_config.json
@@ -0,0 +1,46 @@
+{
+ "channels" : [
+ {
+ "name": "/aos",
+ "type": "aos.timing.Report",
+ "source_node": "test"
+ },
+ {
+ "name": "/aos",
+ "type": "aos.logging.LogMessageFbs",
+ "source_node": "test"
+ },
+ {
+ "name": "/aos",
+ "type": "aos.message_bridge.ClientStatistics",
+ "source_node": "test"
+ },
+ {
+ "name": "/aos",
+ "type": "aos.message_bridge.Timestamp",
+ "source_node": "test"
+ },
+ {
+ "name": "/aos",
+ "type": "aos.message_bridge.ServerStatistics",
+ "source_node": "test"
+ },
+ {
+ "name": "/camera",
+ "type": "frc971.vision.CameraImage",
+ "source_node": "test",
+ "max_size": 10000000
+ },
+ {
+ "name": "/visualize",
+ "type": "foxglove.CompressedImage",
+ "source_node": "test",
+ "max_size": 10000000
+ }
+ ],
+ "nodes": [
+ {
+ "name": "test"
+ }
+ ]
+}
diff --git a/frc971/vision/foxglove_image_converter.cc b/frc971/vision/foxglove_image_converter.cc
index 0c5c736..abde78b 100644
--- a/frc971/vision/foxglove_image_converter.cc
+++ b/frc971/vision/foxglove_image_converter.cc
@@ -3,7 +3,6 @@
#include <opencv2/imgproc.hpp>
namespace frc971::vision {
-namespace {
std::string_view ExtensionForCompression(ImageCompression compression) {
switch (compression) {
case ImageCompression::kJpeg:
@@ -12,25 +11,18 @@
return "png";
}
}
-} // namespace
+
flatbuffers::Offset<foxglove::CompressedImage> CompressImage(
- const CameraImage *raw_image, flatbuffers::FlatBufferBuilder *fbb,
- ImageCompression compression) {
+ const cv::Mat image, const aos::monotonic_clock::time_point eof,
+ flatbuffers::FlatBufferBuilder *fbb, ImageCompression compression) {
std::string_view format = ExtensionForCompression(compression);
// imencode doesn't let us pass in anything other than an std::vector, and
// performance isn't yet a big enough issue to try to avoid the copy.
std::vector<uint8_t> buffer;
- CHECK(raw_image->has_data());
- cv::Mat image_color_mat(cv::Size(raw_image->cols(), raw_image->rows()),
- CV_8UC2, (void *)raw_image->data()->data());
- cv::Mat bgr_image(cv::Size(raw_image->cols(), raw_image->rows()), CV_8UC3);
- cv::cvtColor(image_color_mat, bgr_image, cv::COLOR_YUV2BGR_YUYV);
- CHECK(cv::imencode(absl::StrCat(".", format), bgr_image, buffer));
+ CHECK(cv::imencode(absl::StrCat(".", format), image, buffer));
const flatbuffers::Offset<flatbuffers::Vector<uint8_t>> data_offset =
fbb->CreateVector(buffer);
- const struct timespec timestamp_t =
- aos::time::to_timespec(aos::monotonic_clock::time_point(
- std::chrono::nanoseconds(raw_image->monotonic_timestamp_ns())));
+ const struct timespec timestamp_t = aos::time::to_timespec(eof);
const foxglove::Time time{static_cast<uint32_t>(timestamp_t.tv_sec),
static_cast<uint32_t>(timestamp_t.tv_nsec)};
const flatbuffers::Offset<flatbuffers::String> format_offset =
@@ -47,13 +39,14 @@
std::string_view output_channel,
ImageCompression compression)
: event_loop_(event_loop),
+ image_callback_(
+ event_loop_, input_channel,
+ [this, compression](const cv::Mat image,
+ const aos::monotonic_clock::time_point eof) {
+ auto builder = sender_.MakeBuilder();
+ builder.CheckOk(builder.Send(
+ CompressImage(image, eof, builder.fbb(), compression)));
+ }),
sender_(
- event_loop_->MakeSender<foxglove::CompressedImage>(output_channel)) {
- event_loop_->MakeWatcher(input_channel, [this, compression](
- const CameraImage &image) {
- auto builder = sender_.MakeBuilder();
- builder.CheckOk(
- builder.Send(CompressImage(&image, builder.fbb(), compression)));
- });
-}
+ event_loop_->MakeSender<foxglove::CompressedImage>(output_channel)) {}
} // namespace frc971::vision
diff --git a/frc971/vision/foxglove_image_converter.h b/frc971/vision/foxglove_image_converter.h
index add83a6..872ac14 100644
--- a/frc971/vision/foxglove_image_converter.h
+++ b/frc971/vision/foxglove_image_converter.h
@@ -1,8 +1,9 @@
#ifndef FRC971_VISION_FOXGLOVE_IMAGE_CONVERTER_H_
#define FRC971_VISION_FOXGLOVE_IMAGE_CONVERTER_H_
-#include "external/com_github_foxglove_schemas/CompressedImage_generated.h"
-#include "frc971/vision/vision_generated.h"
#include "aos/events/event_loop.h"
+#include "external/com_github_foxglove_schemas/CompressedImage_generated.h"
+#include "frc971/vision/charuco_lib.h"
+#include "frc971/vision/vision_generated.h"
namespace frc971::vision {
// Empirically, from 2022 logs:
@@ -12,9 +13,11 @@
// conversion with a user-script in Foxglove Studio.
enum class ImageCompression { kJpeg, kPng };
+std::string_view ExtensionForCompression(ImageCompression compression);
+
flatbuffers::Offset<foxglove::CompressedImage> CompressImage(
- const CameraImage *raw_image, flatbuffers::FlatBufferBuilder *fbb,
- ImageCompression compression);
+ const cv::Mat image, const aos::monotonic_clock::time_point eof,
+ flatbuffers::FlatBufferBuilder *fbb, ImageCompression compression);
// This class provides a simple converter that will take an AOS CameraImage
// channel and output
@@ -30,6 +33,7 @@
private:
aos::EventLoop *event_loop_;
+ ImageCallback image_callback_;
aos::Sender<foxglove::CompressedImage> sender_;
};
} // namespace frc971::vision
diff --git a/frc971/vision/foxglove_image_converter_test.cc b/frc971/vision/foxglove_image_converter_test.cc
new file mode 100644
index 0000000..65b3b6b
--- /dev/null
+++ b/frc971/vision/foxglove_image_converter_test.cc
@@ -0,0 +1,68 @@
+#include "frc971/vision/foxglove_image_converter.h"
+
+#include "aos/events/simulated_event_loop.h"
+#include "aos/json_to_flatbuffer.h"
+#include "aos/testing/path.h"
+#include "aos/testing/tmpdir.h"
+#include "gtest/gtest.h"
+
+namespace frc971::vision {
+std::ostream &operator<<(std::ostream &os, ImageCompression compression) {
+ os << ExtensionForCompression(compression);
+ return os;
+}
+namespace testing {
+class ImageConverterTest : public ::testing::TestWithParam<ImageCompression> {
+ protected:
+ ImageConverterTest()
+ : config_(aos::configuration::ReadConfig(
+ aos::testing::ArtifactPath("frc971/vision/converter_config.json"))),
+ factory_(&config_.message()),
+ camera_image_(
+ aos::FileToFlatbuffer<CameraImage>(aos::testing::ArtifactPath(
+ "external/april_tag_test_image/test.bfbs"))),
+ node_(aos::configuration::GetNode(&config_.message(), "test")),
+ test_event_loop_(factory_.MakeEventLoop("test", node_)),
+ image_sender_(test_event_loop_->MakeSender<CameraImage>("/camera")),
+ converter_event_loop_(factory_.MakeEventLoop("converter", node_)),
+ converter_(converter_event_loop_.get(), "/camera", "/visualize",
+ GetParam()),
+ output_path_(absl::StrCat(aos::testing::TestTmpDir(), "/test.",
+ ExtensionForCompression(GetParam()))) {
+ test_event_loop_->OnRun(
+ [this]() { image_sender_.CheckOk(image_sender_.Send(camera_image_)); });
+ test_event_loop_->MakeWatcher(
+ "/visualize", [this](const foxglove::CompressedImage &image) {
+ ASSERT_TRUE(image.has_data());
+ std::string expected_contents =
+ aos::util::ReadFileToStringOrDie(aos::testing::ArtifactPath(
+ absl::StrCat("external/april_tag_test_image/expected.",
+ ExtensionForCompression(GetParam()))));
+ std::string_view data(
+ reinterpret_cast<const char *>(image.data()->data()),
+ image.data()->size());
+ EXPECT_EQ(expected_contents, data);
+ aos::util::WriteStringToFileOrDie(output_path_, data);
+ factory_.Exit();
+ });
+ }
+
+ aos::FlatbufferDetachedBuffer<aos::Configuration> config_;
+ aos::SimulatedEventLoopFactory factory_;
+ aos::FlatbufferVector<CameraImage> camera_image_;
+ const aos::Node *const node_;
+ std::unique_ptr<aos::EventLoop> test_event_loop_;
+ aos::Sender<CameraImage> image_sender_;
+ std::unique_ptr<aos::EventLoop> converter_event_loop_;
+ FoxgloveImageConverter converter_;
+ std::string output_path_;
+};
+
+TEST_P(ImageConverterTest, ImageToFoxglove) { factory_.Run(); }
+
+INSTANTIATE_TEST_SUITE_P(CompressionOptions, ImageConverterTest,
+ ::testing::Values(ImageCompression::kJpeg,
+ ImageCompression::kPng));
+
+} // namespace testing
+} // namespace frc971::vision
diff --git a/frc971/vision/v4l2_reader.cc b/frc971/vision/v4l2_reader.cc
index a6bcb4d..7c0546f 100644
--- a/frc971/vision/v4l2_reader.cc
+++ b/frc971/vision/v4l2_reader.cc
@@ -85,18 +85,23 @@
for (size_t i = 0; i < buffers_.size(); ++i) {
buffers_[i].sender = event_loop_->MakeSender<CameraImage>("/camera");
- EnqueueBuffer(i);
+ MarkBufferToBeEnqueued(i);
}
int type = multiplanar() ? V4L2_BUF_TYPE_VIDEO_CAPTURE_MPLANE
: V4L2_BUF_TYPE_VIDEO_CAPTURE;
PCHECK(Ioctl(VIDIOC_STREAMON, &type) == 0);
}
+void V4L2ReaderBase::MarkBufferToBeEnqueued(int buffer_index) {
+ ReinitializeBuffer(buffer_index);
+ EnqueueBuffer(buffer_index);
+}
+
void V4L2ReaderBase::MaybeEnqueue() {
// First, enqueue any old buffer we already have. This is the one which
// may have been sent.
if (saved_buffer_) {
- EnqueueBuffer(saved_buffer_.index);
+ MarkBufferToBeEnqueued(saved_buffer_.index);
saved_buffer_.Clear();
}
ftrace_.FormatMessage("Enqueued previous buffer %d", saved_buffer_.index);
@@ -114,7 +119,7 @@
// going.
if (previous_buffer) {
ftrace_.FormatMessage("Previous %d", previous_buffer.index);
- EnqueueBuffer(previous_buffer.index);
+ MarkBufferToBeEnqueued(previous_buffer.index);
}
continue;
}
@@ -133,7 +138,12 @@
}
}
-void V4L2ReaderBase::SendLatestImage() { buffers_[saved_buffer_.index].Send(); }
+void V4L2ReaderBase::SendLatestImage() {
+ buffers_[saved_buffer_.index].Send();
+
+ MarkBufferToBeEnqueued(saved_buffer_.index);
+ saved_buffer_.Clear();
+}
void V4L2ReaderBase::SetExposure(size_t duration) {
v4l2_control manual_control;
@@ -236,7 +246,8 @@
CHECK_GE(buffer_number, 0);
CHECK_LT(buffer_number, static_cast<int>(buffers_.size()));
- buffers_[buffer_number].InitializeMessage(ImageSize());
+ CHECK(buffers_[buffer_number].data_pointer != nullptr);
+
struct v4l2_buffer buffer;
struct v4l2_plane planes[1];
memset(&buffer, 0, sizeof(buffer));
@@ -315,16 +326,21 @@
const std::string &image_sensor_subdev)
: V4L2ReaderBase(event_loop, device_name),
epoll_(epoll),
- image_sensor_fd_(open(image_sensor_subdev.c_str(), O_RDWR | O_NONBLOCK)) {
+ image_sensor_fd_(open(image_sensor_subdev.c_str(), O_RDWR | O_NONBLOCK)),
+ buffer_requeuer_([this](int buffer) { EnqueueBuffer(buffer); }, 20) {
PCHECK(image_sensor_fd_.get() != -1)
<< " Failed to open device " << device_name;
-
StreamOn();
epoll_->OnReadable(fd().get(), [this]() { OnImageReady(); });
}
RockchipV4L2Reader::~RockchipV4L2Reader() { epoll_->DeleteFd(fd().get()); }
+void RockchipV4L2Reader::MarkBufferToBeEnqueued(int buffer) {
+ ReinitializeBuffer(buffer);
+ buffer_requeuer_.Push(buffer);
+}
+
void RockchipV4L2Reader::OnImageReady() {
if (!ReadLatestImage()) {
return;
diff --git a/frc971/vision/v4l2_reader.h b/frc971/vision/v4l2_reader.h
index 669c157..31c0888 100644
--- a/frc971/vision/v4l2_reader.h
+++ b/frc971/vision/v4l2_reader.h
@@ -5,10 +5,13 @@
#include <string>
#include "absl/types/span.h"
+#include "aos/containers/ring_buffer.h"
#include "aos/events/epoll.h"
#include "aos/events/event_loop.h"
#include "aos/ftrace.h"
+#include "aos/realtime.h"
#include "aos/scoped/scoped_fd.h"
+#include "aos/util/threaded_consumer.h"
#include "frc971/vision/vision_generated.h"
#include "glog/logging.h"
@@ -57,6 +60,23 @@
void StreamOff();
void StreamOn();
+ // Enqueues a buffer for v4l2 to stream into (expensive).
+ void EnqueueBuffer(int buffer_index);
+
+ // Initializations that need to happen in the main thread.
+ //
+ // Implementations of MarkBufferToBeEnqueued should call this before calling
+ // EnqueueBuffer.
+ void ReinitializeBuffer(int buffer_index) {
+ CHECK_GE(buffer_index, 0);
+ CHECK_LT(buffer_index, static_cast<int>(buffers_.size()));
+ buffers_[buffer_index].InitializeMessage(ImageSize());
+ }
+
+ // Submits a buffer to be enqueued later in a lower priority thread.
+ // Legacy V4L2Reader still does this in the main thread.
+ virtual void MarkBufferToBeEnqueued(int buffer_index);
+
int Ioctl(unsigned long number, void *arg);
bool multiplanar() const { return multiplanar_; }
@@ -70,9 +90,9 @@
const aos::ScopedFD &fd() { return fd_; };
- private:
static constexpr int kNumberBuffers = 4;
+ private:
struct Buffer {
void InitializeMessage(size_t max_image_size);
@@ -113,8 +133,6 @@
// buffer, or BufferInfo() if there wasn't a frame to dequeue.
BufferInfo DequeueBuffer();
- void EnqueueBuffer(int buffer);
-
// The mmaped V4L2 buffers.
std::array<Buffer, kNumberBuffers> buffers_;
@@ -159,11 +177,15 @@
private:
void OnImageReady();
+ void MarkBufferToBeEnqueued(int buffer) override;
+
int ImageSensorIoctl(unsigned long number, void *arg);
aos::internal::EPoll *epoll_;
aos::ScopedFD image_sensor_fd_;
+
+ aos::util::ThreadedConsumer<int, kNumberBuffers> buffer_requeuer_;
};
} // namespace vision
diff --git a/y2022/vision/calibrate_extrinsics.cc b/y2022/vision/calibrate_extrinsics.cc
index b5005fe..04b24ea 100644
--- a/y2022/vision/calibrate_extrinsics.cc
+++ b/y2022/vision/calibrate_extrinsics.cc
@@ -2,6 +2,7 @@
#include "Eigen/Geometry"
#include "absl/strings/str_format.h"
#include "aos/events/logging/log_reader.h"
+#include "aos/events/logging/log_writer.h"
#include "aos/init.h"
#include "aos/network/team_number.h"
#include "aos/time/time.h"
@@ -21,6 +22,8 @@
DEFINE_string(target_type, "charuco",
"Type of target: aruco|charuco|charuco_diamond");
DEFINE_string(image_channel, "/camera", "Channel to listen for images on");
+DEFINE_string(output_logs, "/tmp/calibration/",
+ "Output folder for visualization logs.");
namespace frc971 {
namespace vision {
@@ -33,11 +36,24 @@
void Main(int argc, char **argv) {
CalibrationData data;
+ std::optional<uint16_t> pi_number = aos::network::ParsePiNumber(FLAGS_pi);
+ CHECK(pi_number);
+ const std::string pi_name = absl::StrCat("pi", *pi_number);
+ LOG(INFO) << "Pi " << *pi_number;
+ aos::FlatbufferDetachedBuffer<aos::Configuration> config = [argc, argv,
+ pi_name]() {
+ aos::logger::LogReader reader(
+ aos::logger::SortParts(aos::logger::FindLogs(argc, argv)));
+ return CalibrationFoxgloveVisualizer::AddVisualizationChannels(
+ reader.logged_configuration(),
+ aos::configuration::GetNode(reader.logged_configuration(), pi_name));
+ }();
{
// Now, accumulate all the data into the data object.
aos::logger::LogReader reader(
- aos::logger::SortParts(aos::logger::FindLogs(argc, argv)));
+ aos::logger::SortParts(aos::logger::FindLogs(argc, argv)),
+ &config.message());
aos::SimulatedEventLoopFactory factory(reader.configuration());
reader.Register(&factory);
@@ -50,11 +66,8 @@
const aos::Node *const roborio_node =
aos::configuration::GetNode(factory.configuration(), "roborio");
- std::optional<uint16_t> pi_number = aos::network::ParsePiNumber(FLAGS_pi);
- CHECK(pi_number);
- LOG(INFO) << "Pi " << *pi_number;
- const aos::Node *const pi_node = aos::configuration::GetNode(
- factory.configuration(), absl::StrCat("pi", *pi_number));
+ const aos::Node *const pi_node =
+ aos::configuration::GetNode(factory.configuration(), pi_name);
LOG(INFO) << "imu " << aos::FlatbufferToJson(imu_node);
LOG(INFO) << "roboRIO " << aos::FlatbufferToJson(roborio_node);
@@ -67,6 +80,11 @@
std::unique_ptr<aos::EventLoop> pi_event_loop =
factory.MakeEventLoop("calibration", pi_node);
+ std::unique_ptr<aos::EventLoop> logger_loop =
+ factory.MakeEventLoop("logger", pi_node);
+ aos::logger::Logger logger(logger_loop.get());
+ logger.StartLoggingOnRun(FLAGS_output_logs);
+
TargetType target_type = TargetType::kCharuco;
if (FLAGS_target_type == "aruco") {
target_type = TargetType::kAruco;
diff --git a/y2023/control_loops/python/BUILD b/y2023/control_loops/python/BUILD
index b024676..a2ea6d8 100644
--- a/y2023/control_loops/python/BUILD
+++ b/y2023/control_loops/python/BUILD
@@ -31,6 +31,24 @@
],
)
+py_binary(
+ name = "graph_edit",
+ srcs = [
+ "graph_edit.py",
+ "graph_generate.py",
+ ],
+ legacy_create_init = False,
+ target_compatible_with = ["@platforms//cpu:x86_64"],
+ deps = [
+ ":python_init",
+ "//frc971/control_loops/python:basic_window",
+ "//frc971/control_loops/python:color",
+ "@pip//numpy",
+ "@pip//pygobject",
+ "@pip//shapely",
+ ],
+)
+
py_library(
name = "polydrivetrain_lib",
srcs = [
diff --git a/y2023/control_loops/python/graph_edit.py b/y2023/control_loops/python/graph_edit.py
new file mode 100644
index 0000000..7b6179c
--- /dev/null
+++ b/y2023/control_loops/python/graph_edit.py
@@ -0,0 +1,495 @@
+#!/usr/bin/python3
+
+from __future__ import print_function
+import os
+from frc971.control_loops.python import basic_window
+from frc971.control_loops.python.color import Color, palette
+import random
+import gi
+import numpy
+
+gi.require_version('Gtk', '3.0')
+from gi.repository import Gdk, Gtk
+import cairo
+import graph_generate
+from graph_generate import XYSegment, AngleSegment, to_theta, to_xy, alpha_blend
+from graph_generate import back_to_xy_loop, subdivide_theta, to_theta_loop
+from graph_generate import l1, l2, joint_center
+
+from frc971.control_loops.python.basic_window import OverrideMatrix, identity, quit_main_loop, set_color
+
+import shapely
+from shapely.geometry import Polygon
+
+
+def px(cr):
+ return OverrideMatrix(cr, identity)
+
+
+def draw_px_cross(cr, length_px):
+ """Draws a cross with fixed dimensions in pixel space."""
+ with px(cr):
+ x, y = cr.get_current_point()
+ cr.move_to(x, y - length_px)
+ cr.line_to(x, y + length_px)
+ cr.stroke()
+
+ cr.move_to(x - length_px, y)
+ cr.line_to(x + length_px, y)
+ cr.stroke()
+
+
+def angle_dist_sqr(a1, a2):
+ """Distance between two points in angle space."""
+ return (a1[0] - a2[0])**2 + (a1[1] - a2[1])**2
+
+
+# Find the highest y position that intersects the vertical line defined by x.
+def inter_y(x):
+ return numpy.sqrt((l2 + l1)**2 -
+ (x - joint_center[0])**2) + joint_center[1]
+
+
+# This is the x position where the inner (hyperextension) circle intersects the horizontal line
+derr = numpy.sqrt((l1 - l2)**2 - (joint_center[1] - 0.3048)**2)
+
+
+# Define min and max l1 angles based on vertical constraints.
+def get_angle(boundary):
+ h = numpy.sqrt((l1)**2 - (boundary - joint_center[0])**2) + joint_center[1]
+ return numpy.arctan2(h, boundary - joint_center[0])
+
+
+# left hand side lines
+lines1 = [
+ (-0.826135, inter_y(-0.826135)),
+ (-0.826135, 0.1397),
+ (-23.025 * 0.0254, 0.1397),
+ (-23.025 * 0.0254, 0.3048),
+ (joint_center[0] - derr, 0.3048),
+]
+
+# right hand side lines
+lines2 = [(joint_center[0] + derr, 0.3048), (0.422275, 0.3048),
+ (0.422275, 0.1397), (0.826135, 0.1397),
+ (0.826135, inter_y(0.826135))]
+
+t1_min = get_angle((32.525 - 4.0) * 0.0254)
+t2_min = -7.0 / 4.0 * numpy.pi
+
+t1_max = get_angle((-32.525 + 4.0) * 0.0254)
+t2_max = numpy.pi * 3.0 / 4.0
+
+
+# Draw lines to cr + stroke.
+def draw_lines(cr, lines):
+ cr.move_to(lines[0][0], lines[0][1])
+ for pt in lines[1:]:
+ cr.line_to(pt[0], pt[1])
+ with px(cr):
+ cr.stroke()
+
+
+# Rotate a rasterized loop such that it aligns to when the parameters loop
+def rotate_to_jump_point(points):
+ last_pt = points[0]
+ for pt_i in range(1, len(points)):
+ pt = points[pt_i]
+ delta = last_pt[1] - pt[1]
+ if abs(delta) > numpy.pi:
+ return points[pt_i:] + points[:pt_i]
+ last_pt = pt
+ return points
+
+
+# shift points vertically by dy.
+def y_shift(points, dy):
+ return [(x, y + dy) for x, y in points]
+
+
+lines1_theta_part = rotate_to_jump_point(to_theta_loop(lines1, 0))
+lines2_theta_part = rotate_to_jump_point(to_theta_loop(lines2))
+
+# Some hacks here to make a single polygon by shifting to get an extra copy of the contraints.
+lines1_theta = y_shift(lines1_theta_part, -numpy.pi * 2) + lines1_theta_part + \
+ y_shift(lines1_theta_part, numpy.pi * 2)
+lines2_theta = y_shift(lines2_theta_part, numpy.pi * 2) + lines2_theta_part + \
+ y_shift(lines2_theta_part, -numpy.pi * 2)
+
+lines_theta = lines1_theta + lines2_theta
+
+p1 = Polygon(lines_theta)
+
+p2 = Polygon([(t1_min, t2_min), (t1_max, t2_min), (t1_max, t2_max),
+ (t1_min, t2_max)])
+
+# Fully computed theta constrints.
+lines_theta = list(p1.intersection(p2).exterior.coords)
+
+lines1_theta_back = back_to_xy_loop(lines1_theta)
+lines2_theta_back = back_to_xy_loop(lines2_theta)
+
+lines_theta_back = back_to_xy_loop(lines_theta)
+
+
+# Get the closest point to a line from a test pt.
+def get_closest(prev, cur, pt):
+ dx_ang = (cur[0] - prev[0])
+ dy_ang = (cur[1] - prev[1])
+
+ d = numpy.sqrt(dx_ang**2 + dy_ang**2)
+ if (d < 0.000001):
+ return prev, numpy.sqrt((prev[0] - pt[0])**2 + (prev[1] - pt[1])**2)
+
+ pdx = -dy_ang / d
+ pdy = dx_ang / d
+
+ dpx = pt[0] - prev[0]
+ dpy = pt[1] - prev[1]
+
+ alpha = (dx_ang * dpx + dy_ang * dpy) / d / d
+
+ if (alpha < 0):
+ return prev, numpy.sqrt((prev[0] - pt[0])**2 + (prev[1] - pt[1])**2)
+ elif (alpha > 1):
+ return cur, numpy.sqrt((cur[0] - pt[0])**2 + (cur[1] - pt[1])**2)
+ else:
+ return (alpha_blend(prev[0], cur[0], alpha), alpha_blend(prev[1], cur[1], alpha)), \
+ abs(dpx * pdx + dpy * pdy)
+
+
+def closest_segment(lines, pt):
+ c_pt, c_pt_dist = get_closest(lines[-1], lines[0], pt)
+ for i in range(1, len(lines)):
+ prev = lines[i - 1]
+ cur = lines[i]
+ c_pt_new, c_pt_new_dist = get_closest(prev, cur, pt)
+ if c_pt_new_dist < c_pt_dist:
+ c_pt = c_pt_new
+ c_pt_dist = c_pt_new_dist
+ return c_pt, c_pt_dist
+
+
+# Create a GTK+ widget on which we will draw using Cairo
+class Silly(basic_window.BaseWindow):
+
+ def __init__(self):
+ super(Silly, self).__init__()
+
+ self.window = Gtk.Window()
+ self.window.set_title("DrawingArea")
+
+ self.window.set_events(Gdk.EventMask.BUTTON_PRESS_MASK
+ | Gdk.EventMask.BUTTON_RELEASE_MASK
+ | Gdk.EventMask.POINTER_MOTION_MASK
+ | Gdk.EventMask.SCROLL_MASK
+ | Gdk.EventMask.KEY_PRESS_MASK)
+ self.method_connect("key-press-event", self.do_key_press)
+ self.method_connect("button-press-event",
+ self._do_button_press_internal)
+ self.method_connect("configure-event", self._do_configure)
+ self.window.add(self)
+ self.window.show_all()
+
+ self.theta_version = False
+ self.reinit_extents()
+
+ self.last_pos = (numpy.pi / 2.0, 1.0)
+ self.circular_index_select = -1
+
+ # Extra stuff for drawing lines.
+ self.segments = []
+ self.prev_segment_pt = None
+ self.now_segment_pt = None
+ self.spline_edit = 0
+ self.edit_control1 = True
+
+ def do_key_press(self, event):
+ pass
+
+ def _do_button_press_internal(self, event):
+ o_x = event.x
+ o_y = event.y
+ x = event.x - self.window_shape[0] / 2
+ y = self.window_shape[1] / 2 - event.y
+ scale = self.get_current_scale()
+ event.x = x / scale + self.center[0]
+ event.y = y / scale + self.center[1]
+ self.do_button_press(event)
+ event.x = o_x
+ event.y = o_y
+
+ def do_button_press(self, event):
+ pass
+
+ def _do_configure(self, event):
+ self.window_shape = (event.width, event.height)
+
+ def redraw(self):
+ if not self.needs_redraw:
+ self.needs_redraw = True
+ self.window.queue_draw()
+
+ def method_connect(self, event, cb):
+
+ def handler(obj, *args):
+ cb(*args)
+
+ self.window.connect(event, handler)
+
+ def reinit_extents(self):
+ if self.theta_version:
+ self.extents_x_min = -numpy.pi * 2
+ self.extents_x_max = numpy.pi * 2
+ self.extents_y_min = -numpy.pi * 2
+ self.extents_y_max = numpy.pi * 2
+ else:
+ self.extents_x_min = -40.0 * 0.0254
+ self.extents_x_max = 40.0 * 0.0254
+ self.extents_y_min = -4.0 * 0.0254
+ self.extents_y_max = 110.0 * 0.0254
+
+ self.init_extents(
+ (0.5 * (self.extents_x_min + self.extents_x_max), 0.5 *
+ (self.extents_y_max + self.extents_y_min)),
+ (1.0 * (self.extents_x_max - self.extents_x_min), 1.0 *
+ (self.extents_y_max - self.extents_y_min)))
+
+ # Handle the expose-event by drawing
+ def handle_draw(self, cr):
+ # use "with px(cr): blah;" to transform to pixel coordinates.
+
+ # Fill the background color of the window with grey
+ set_color(cr, palette["GREY"])
+ cr.paint()
+
+ # Draw a extents rectangle
+ set_color(cr, palette["WHITE"])
+ cr.rectangle(self.extents_x_min, self.extents_y_min,
+ (self.extents_x_max - self.extents_x_min),
+ self.extents_y_max - self.extents_y_min)
+ cr.fill()
+
+ if not self.theta_version:
+ # Draw a filled white rectangle.
+ set_color(cr, palette["WHITE"])
+ cr.rectangle(-2.0, -2.0, 4.0, 4.0)
+ cr.fill()
+
+ set_color(cr, palette["BLUE"])
+ cr.arc(joint_center[0], joint_center[1], l2 + l1, 0,
+ 2.0 * numpy.pi)
+ with px(cr):
+ cr.stroke()
+ cr.arc(joint_center[0], joint_center[1], l1 - l2, 0,
+ 2.0 * numpy.pi)
+ with px(cr):
+ cr.stroke()
+ else:
+ # Draw a filled white rectangle.
+ set_color(cr, palette["WHITE"])
+ cr.rectangle(-numpy.pi, -numpy.pi, numpy.pi * 2.0, numpy.pi * 2.0)
+ cr.fill()
+
+ if self.theta_version:
+ set_color(cr, palette["BLUE"])
+ for i in range(-6, 6):
+ cr.move_to(-40, -40 + i * numpy.pi)
+ cr.line_to(40, 40 + i * numpy.pi)
+ with px(cr):
+ cr.stroke()
+
+ if self.theta_version:
+ set_color(cr, Color(0.5, 0.5, 1.0))
+ draw_lines(cr, lines_theta)
+ else:
+ set_color(cr, Color(0.5, 1.0, 1.0))
+ draw_lines(cr, lines1)
+ draw_lines(cr, lines2)
+
+ def get_circular_index(pt):
+ theta1, theta2 = pt
+ circular_index = int(numpy.floor((theta2 - theta1) / numpy.pi))
+ return circular_index
+
+ set_color(cr, palette["BLUE"])
+ lines = subdivide_theta(lines_theta)
+ o_circular_index = circular_index = get_circular_index(lines[0])
+ p_xy = to_xy(lines[0][0], lines[0][1])
+ if circular_index == self.circular_index_select:
+ cr.move_to(p_xy[0] + circular_index * 0, p_xy[1])
+ for pt in lines[1:]:
+ p_xy = to_xy(pt[0], pt[1])
+ circular_index = get_circular_index(pt)
+ if o_circular_index == self.circular_index_select:
+ cr.line_to(p_xy[0] + o_circular_index * 0, p_xy[1])
+ if circular_index != o_circular_index:
+ o_circular_index = circular_index
+ with px(cr):
+ cr.stroke()
+ if circular_index == self.circular_index_select:
+ cr.move_to(p_xy[0] + circular_index * 0, p_xy[1])
+
+ with px(cr):
+ cr.stroke()
+
+ if not self.theta_version:
+ theta1, theta2 = to_theta(self.last_pos,
+ self.circular_index_select)
+ x, y = joint_center[0], joint_center[1]
+ cr.move_to(x, y)
+
+ x += numpy.cos(theta1) * l1
+ y += numpy.sin(theta1) * l1
+ cr.line_to(x, y)
+ x += numpy.cos(theta2) * l2
+ y += numpy.sin(theta2) * l2
+ cr.line_to(x, y)
+ with px(cr):
+ cr.stroke()
+
+ cr.move_to(self.last_pos[0], self.last_pos[1])
+ set_color(cr, Color(0.0, 1.0, 0.2))
+ draw_px_cross(cr, 20)
+
+ if self.theta_version:
+ set_color(cr, Color(0.0, 1.0, 0.2))
+ cr.move_to(self.last_pos[0], self.last_pos[1])
+ draw_px_cross(cr, 5)
+
+ c_pt, dist = closest_segment(lines_theta, self.last_pos)
+ print("dist:", dist, c_pt, self.last_pos)
+ set_color(cr, palette["CYAN"])
+ cr.move_to(c_pt[0], c_pt[1])
+ draw_px_cross(cr, 5)
+
+ set_color(cr, Color(0.0, 0.5, 1.0))
+ for segment in self.segments:
+ color = [0, random.random(), 1]
+ random.shuffle(color)
+ set_color(cr, Color(color[0], color[1], color[2]))
+ segment.DrawTo(cr, self.theta_version)
+ with px(cr):
+ cr.stroke()
+
+ set_color(cr, Color(0.0, 1.0, 0.5))
+ segment = self.current_seg()
+ if segment:
+ print(segment)
+ segment.DrawTo(cr, self.theta_version)
+ with px(cr):
+ cr.stroke()
+
+ def cur_pt_in_theta(self):
+ if self.theta_version: return self.last_pos
+ return to_theta(self.last_pos, self.circular_index_select)
+
+ # Current segment based on which mode the drawing system is in.
+ def current_seg(self):
+ if self.prev_segment_pt and self.now_segment_pt:
+ if self.theta_version:
+ return AngleSegment(self.prev_segment_pt, self.now_segment_pt)
+ else:
+ return XYSegment(self.prev_segment_pt, self.now_segment_pt)
+
+ def do_key_press(self, event):
+ keyval = Gdk.keyval_to_lower(event.keyval)
+ print("Gdk.KEY_" + Gdk.keyval_name(keyval))
+ if keyval == Gdk.KEY_q:
+ print("Found q key and exiting.")
+ quit_main_loop()
+ elif keyval == Gdk.KEY_c:
+ # Increment which arm solution we render
+ self.circular_index_select += 1
+ print(self.circular_index_select)
+ elif keyval == Gdk.KEY_v:
+ # Decrement which arm solution we render
+ self.circular_index_select -= 1
+ print(self.circular_index_select)
+ elif keyval == Gdk.KEY_w:
+ # Add this segment to the segment list.
+ segment = self.current_seg()
+ if segment: self.segments.append(segment)
+ self.prev_segment_pt = self.now_segment_pt
+
+ elif keyval == Gdk.KEY_r:
+ self.prev_segment_pt = self.now_segment_pt
+
+ elif keyval == Gdk.KEY_p:
+ # Print out the segments.
+ print(repr(self.segments))
+ elif keyval == Gdk.KEY_g:
+ # Generate theta points.
+ if self.segments:
+ print(repr(self.segments[0].ToThetaPoints()))
+ elif keyval == Gdk.KEY_e:
+ best_pt = self.now_segment_pt
+ best_dist = 1e10
+ for segment in self.segments:
+ d = angle_dist_sqr(segment.start, self.now_segment_pt)
+ if (d < best_dist):
+ best_pt = segment.start
+ best_dist = d
+ d = angle_dist_sqr(segment.end, self.now_segment_pt)
+ if (d < best_dist):
+ best_pt = segment.end
+ best_dist = d
+ self.now_segment_pt = best_pt
+
+ elif keyval == Gdk.KEY_t:
+ # Toggle between theta and xy renderings
+ if self.theta_version:
+ theta1, theta2 = self.last_pos
+ data = to_xy(theta1, theta2)
+ self.circular_index_select = int(
+ numpy.floor((theta2 - theta1) / numpy.pi))
+ self.last_pos = (data[0], data[1])
+ else:
+ self.last_pos = self.cur_pt_in_theta()
+
+ self.theta_version = not self.theta_version
+ self.reinit_extents()
+
+ elif keyval == Gdk.KEY_z:
+ self.edit_control1 = not self.edit_control1
+ if self.edit_control1:
+ self.now_segment_pt = self.segments[0].control1
+ else:
+ self.now_segment_pt = self.segments[0].control2
+ if not self.theta_version:
+ data = to_xy(self.now_segment_pt[0], self.now_segment_pt[1])
+ self.last_pos = (data[0], data[1])
+ else:
+ self.last_pos = self.now_segment_pt
+
+ print("self.last_pos: ", self.last_pos, " ci: ",
+ self.circular_index_select)
+
+ self.redraw()
+
+ def do_button_press(self, event):
+ self.last_pos = (event.x, event.y)
+ self.now_segment_pt = self.cur_pt_in_theta()
+
+ if self.edit_control1:
+ self.segments[0].control1 = self.now_segment_pt
+ else:
+ self.segments[0].control2 = self.now_segment_pt
+
+ print('Clicked at theta: %s' % (repr(self.now_segment_pt, )))
+ if not self.theta_version:
+ print('Clicked at xy, circular index: (%f, %f, %f)' %
+ (self.last_pos[0], self.last_pos[1],
+ self.circular_index_select))
+
+ print('c1: numpy.array([%f, %f])' %
+ (self.segments[0].control1[0], self.segments[0].control1[1]))
+ print('c2: numpy.array([%f, %f])' %
+ (self.segments[0].control2[0], self.segments[0].control2[1]))
+
+ self.redraw()
+
+
+silly = Silly()
+silly.segments = graph_generate.segments
+basic_window.RunApp()
diff --git a/y2023/control_loops/python/graph_generate.py b/y2023/control_loops/python/graph_generate.py
new file mode 100644
index 0000000..046b9dd
--- /dev/null
+++ b/y2023/control_loops/python/graph_generate.py
@@ -0,0 +1,798 @@
+import numpy
+
+# joint_center in x-y space.
+joint_center = (-0.299, 0.299)
+
+# Joint distances (l1 = "proximal", l2 = "distal")
+l1 = 46.25 * 0.0254
+l2 = 43.75 * 0.0254
+
+
+# Convert from x-y coordinates to theta coordinates.
+# orientation is a bool. This orientation is circular_index mod 2.
+# where circular_index is the circular index, or the position in the
+# "hyperextension" zones. "cross_point" allows shifting the place where
+# it rounds the result so that it draws nicer (no other functional differences).
+def to_theta(pt, circular_index, cross_point=-numpy.pi):
+ orient = (circular_index % 2) == 0
+ x = pt[0]
+ y = pt[1]
+ x -= joint_center[0]
+ y -= joint_center[1]
+ l3 = numpy.hypot(x, y)
+ t3 = numpy.arctan2(y, x)
+ theta1 = numpy.arccos((l1**2 + l3**2 - l2**2) / (2 * l1 * l3))
+
+ if orient:
+ theta1 = -theta1
+ theta1 += t3
+ theta1 = (theta1 - cross_point) % (2 * numpy.pi) + cross_point
+ theta2 = numpy.arctan2(y - l1 * numpy.sin(theta1),
+ x - l1 * numpy.cos(theta1))
+ return numpy.array((theta1, theta2))
+
+
+# Simple trig to go back from theta1, theta2 to x-y
+def to_xy(theta1, theta2):
+ x = numpy.cos(theta1) * l1 + numpy.cos(theta2) * l2 + joint_center[0]
+ y = numpy.sin(theta1) * l1 + numpy.sin(theta2) * l2 + joint_center[1]
+ orient = ((theta2 - theta1) % (2.0 * numpy.pi)) < numpy.pi
+ return (x, y, orient)
+
+
+def get_circular_index(theta):
+ return int(numpy.floor((theta[1] - theta[0]) / numpy.pi))
+
+
+def get_xy(theta):
+ theta1 = theta[0]
+ theta2 = theta[1]
+ x = numpy.cos(theta1) * l1 + numpy.cos(theta2) * l2 + joint_center[0]
+ y = numpy.sin(theta1) * l1 + numpy.sin(theta2) * l2 + joint_center[1]
+ return numpy.array((x, y))
+
+
+# Draw a list of lines to a cairo context.
+def draw_lines(cr, lines):
+ cr.move_to(lines[0][0], lines[0][1])
+ for pt in lines[1:]:
+ cr.line_to(pt[0], pt[1])
+
+
+max_dist = 0.01
+max_dist_theta = numpy.pi / 64
+xy_end_circle_size = 0.01
+theta_end_circle_size = 0.07
+
+
+# Subdivide in theta space.
+def subdivide_theta(lines):
+ out = []
+ last_pt = lines[0]
+ out.append(last_pt)
+ for n_pt in lines[1:]:
+ for pt in subdivide(last_pt, n_pt, max_dist_theta):
+ out.append(pt)
+ last_pt = n_pt
+
+ return out
+
+
+# subdivide in xy space.
+def subdivide_xy(lines, max_dist=max_dist):
+ out = []
+ last_pt = lines[0]
+ out.append(last_pt)
+ for n_pt in lines[1:]:
+ for pt in subdivide(last_pt, n_pt, max_dist):
+ out.append(pt)
+ last_pt = n_pt
+
+ return out
+
+
+def to_theta_with_ci(pt, circular_index):
+ return to_theta_with_circular_index(pt[0], pt[1], circular_index)
+
+
+# to_theta, but distinguishes between
+def to_theta_with_circular_index(x, y, circular_index):
+ theta1, theta2 = to_theta((x, y), circular_index)
+ n_circular_index = int(numpy.floor((theta2 - theta1) / numpy.pi))
+ theta2 = theta2 + ((circular_index - n_circular_index)) * numpy.pi
+ return numpy.array((theta1, theta2))
+
+
+# alpha is in [0, 1] and is the weight to merge a and b.
+def alpha_blend(a, b, alpha):
+ """Blends a and b.
+
+ Args:
+ alpha: double, Ratio. Needs to be in [0, 1] and is the weight to blend a
+ and b.
+ """
+ return b * alpha + (1.0 - alpha) * a
+
+
+def normalize(v):
+ """Normalize a vector while handling 0 length vectors."""
+ norm = numpy.linalg.norm(v)
+ if norm == 0:
+ return v
+ return v / norm
+
+
+# CI is circular index and allows selecting between all the stats that map
+# to the same x-y state (by giving them an integer index).
+# This will compute approximate first and second derivatives with respect
+# to path length.
+def to_theta_with_circular_index_and_derivs(x, y, dx, dy,
+ circular_index_select):
+ a = to_theta_with_circular_index(x, y, circular_index_select)
+ b = to_theta_with_circular_index(x + dx * 0.0001, y + dy * 0.0001,
+ circular_index_select)
+ c = to_theta_with_circular_index(x - dx * 0.0001, y - dy * 0.0001,
+ circular_index_select)
+ d1 = normalize(b - a)
+ d2 = normalize(c - a)
+ accel = (d1 + d2) / numpy.linalg.norm(a - b)
+ return (a[0], a[1], d1[0], d1[1], accel[0], accel[1])
+
+
+def to_theta_with_ci_and_derivs(p_prev, p, p_next, c_i_select):
+ a = to_theta(p, c_i_select)
+ b = to_theta(p_next, c_i_select)
+ c = to_theta(p_prev, c_i_select)
+ d1 = normalize(b - a)
+ d2 = normalize(c - a)
+ accel = (d1 + d2) / numpy.linalg.norm(a - b)
+ return (a[0], a[1], d1[0], d1[1], accel[0], accel[1])
+
+
+# Generic subdivision algorithm.
+def subdivide(p1, p2, max_dist):
+ dx = p2[0] - p1[0]
+ dy = p2[1] - p1[1]
+ dist = numpy.sqrt(dx**2 + dy**2)
+ n = int(numpy.ceil(dist / max_dist))
+ return [(alpha_blend(p1[0], p2[0],
+ float(i) / n), alpha_blend(p1[1], p2[1],
+ float(i) / n))
+ for i in range(1, n + 1)]
+
+
+# convert from an xy space loop into a theta loop.
+# All segements are expected go from one "hyper-extension" boundary
+# to another, thus we must go backwards over the "loop" to get a loop in
+# x-y space.
+def to_theta_loop(lines, cross_point=-numpy.pi):
+ out = []
+ last_pt = lines[0]
+ for n_pt in lines[1:]:
+ for pt in subdivide(last_pt, n_pt, max_dist):
+ out.append(to_theta(pt, 0, cross_point))
+ last_pt = n_pt
+ for n_pt in reversed(lines[:-1]):
+ for pt in subdivide(last_pt, n_pt, max_dist):
+ out.append(to_theta(pt, 1, cross_point))
+ last_pt = n_pt
+ return out
+
+
+# Convert a loop (list of line segments) into
+# The name incorrectly suggests that it is cyclic.
+def back_to_xy_loop(lines):
+ out = []
+ last_pt = lines[0]
+ out.append(to_xy(last_pt[0], last_pt[1]))
+ for n_pt in lines[1:]:
+ for pt in subdivide(last_pt, n_pt, max_dist_theta):
+ out.append(to_xy(pt[0], pt[1]))
+ last_pt = n_pt
+
+ return out
+
+
+# Segment in angle space.
+class AngleSegment:
+
+ def __init__(self, start, end, name=None, alpha_unitizer=None, vmax=None):
+ """Creates an angle segment.
+
+ Args:
+ start: (double, double), The start of the segment in theta1, theta2
+ coordinates in radians
+ end: (double, double), The end of the segment in theta1, theta2
+ coordinates in radians
+ """
+ self.start = start
+ self.end = end
+ self.name = name
+ self.alpha_unitizer = alpha_unitizer
+ self.vmax = vmax
+
+ def __repr__(self):
+ return "AngleSegment(%s, %s)" % (repr(self.start), repr(self.end))
+
+ def DrawTo(self, cr, theta_version):
+ if theta_version:
+ cr.move_to(self.start[0], self.start[1] + theta_end_circle_size)
+ cr.arc(self.start[0], self.start[1], theta_end_circle_size, 0,
+ 2.0 * numpy.pi)
+ cr.move_to(self.end[0], self.end[1] + theta_end_circle_size)
+ cr.arc(self.end[0], self.end[1], theta_end_circle_size, 0,
+ 2.0 * numpy.pi)
+ cr.move_to(self.start[0], self.start[1])
+ cr.line_to(self.end[0], self.end[1])
+ else:
+ start_xy = to_xy(self.start[0], self.start[1])
+ end_xy = to_xy(self.end[0], self.end[1])
+ draw_lines(cr, back_to_xy_loop([self.start, self.end]))
+ cr.move_to(start_xy[0] + xy_end_circle_size, start_xy[1])
+ cr.arc(start_xy[0], start_xy[1], xy_end_circle_size, 0,
+ 2.0 * numpy.pi)
+ cr.move_to(end_xy[0] + xy_end_circle_size, end_xy[1])
+ cr.arc(end_xy[0], end_xy[1], xy_end_circle_size, 0, 2.0 * numpy.pi)
+
+ def ToThetaPoints(self):
+ dx = self.end[0] - self.start[0]
+ dy = self.end[1] - self.start[1]
+ mag = numpy.hypot(dx, dy)
+ dx /= mag
+ dy /= mag
+
+ return [(self.start[0], self.start[1], dx, dy, 0.0, 0.0),
+ (self.end[0], self.end[1], dx, dy, 0.0, 0.0)]
+
+
+class XYSegment:
+ """Straight line in XY space."""
+
+ def __init__(self, start, end, name=None, alpha_unitizer=None, vmax=None):
+ """Creates an XY segment.
+
+ Args:
+ start: (double, double), The start of the segment in theta1, theta2
+ coordinates in radians
+ end: (double, double), The end of the segment in theta1, theta2
+ coordinates in radians
+ """
+ self.start = start
+ self.end = end
+ self.name = name
+ self.alpha_unitizer = alpha_unitizer
+ self.vmax = vmax
+
+ def __repr__(self):
+ return "XYSegment(%s, %s)" % (repr(self.start), repr(self.end))
+
+ def DrawTo(self, cr, theta_version):
+ if theta_version:
+ theta1, theta2 = self.start
+ circular_index_select = int(
+ numpy.floor((self.start[1] - self.start[0]) / numpy.pi))
+ start = get_xy(self.start)
+ end = get_xy(self.end)
+
+ ln = [(start[0], start[1]), (end[0], end[1])]
+ draw_lines(cr, [
+ to_theta_with_circular_index(x, y, circular_index_select)
+ for x, y in subdivide_xy(ln)
+ ])
+ cr.move_to(self.start[0] + theta_end_circle_size, self.start[1])
+ cr.arc(self.start[0], self.start[1], theta_end_circle_size, 0,
+ 2.0 * numpy.pi)
+ cr.move_to(self.end[0] + theta_end_circle_size, self.end[1])
+ cr.arc(self.end[0], self.end[1], theta_end_circle_size, 0,
+ 2.0 * numpy.pi)
+ else:
+ start = get_xy(self.start)
+ end = get_xy(self.end)
+ cr.move_to(start[0], start[1])
+ cr.line_to(end[0], end[1])
+ cr.move_to(start[0] + xy_end_circle_size, start[1])
+ cr.arc(start[0], start[1], xy_end_circle_size, 0, 2.0 * numpy.pi)
+ cr.move_to(end[0] + xy_end_circle_size, end[1])
+ cr.arc(end[0], end[1], xy_end_circle_size, 0, 2.0 * numpy.pi)
+
+ def ToThetaPoints(self):
+ """ Converts to points in theta space via to_theta_with_circular_index_and_derivs"""
+ theta1, theta2 = self.start
+ circular_index_select = int(
+ numpy.floor((self.start[1] - self.start[0]) / numpy.pi))
+ start = get_xy(self.start)
+ end = get_xy(self.end)
+
+ ln = [(start[0], start[1]), (end[0], end[1])]
+
+ dx = end[0] - start[0]
+ dy = end[1] - start[1]
+ mag = numpy.hypot(dx, dy)
+ dx /= mag
+ dy /= mag
+
+ return [
+ to_theta_with_circular_index_and_derivs(x, y, dx, dy,
+ circular_index_select)
+ for x, y in subdivide_xy(ln, 0.01)
+ ]
+
+
+def spline_eval(start, control1, control2, end, alpha):
+ a = alpha_blend(start, control1, alpha)
+ b = alpha_blend(control1, control2, alpha)
+ c = alpha_blend(control2, end, alpha)
+ return alpha_blend(alpha_blend(a, b, alpha), alpha_blend(b, c, alpha),
+ alpha)
+
+
+def subdivide_spline(start, control1, control2, end):
+ # TODO: pick N based on spline parameters? or otherwise change it to be more evenly spaced?
+ n = 100
+ for i in range(0, n + 1):
+ yield i / float(n)
+
+
+class SplineSegment:
+
+ def __init__(self,
+ start,
+ control1,
+ control2,
+ end,
+ name=None,
+ alpha_unitizer=None,
+ vmax=None):
+ self.start = start
+ self.control1 = control1
+ self.control2 = control2
+ self.end = end
+ self.name = name
+ self.alpha_unitizer = alpha_unitizer
+ self.vmax = vmax
+
+ def __repr__(self):
+ return "SplineSegment(%s, %s, %s, %s)" % (repr(
+ self.start), repr(self.control1), repr(
+ self.control2), repr(self.end))
+
+ def DrawTo(self, cr, theta_version):
+ if theta_version:
+ c_i_select = get_circular_index(self.start)
+ start = get_xy(self.start)
+ control1 = get_xy(self.control1)
+ control2 = get_xy(self.control2)
+ end = get_xy(self.end)
+
+ draw_lines(cr, [
+ to_theta(spline_eval(start, control1, control2, end, alpha),
+ c_i_select)
+ for alpha in subdivide_spline(start, control1, control2, end)
+ ])
+ cr.move_to(self.start[0] + theta_end_circle_size, self.start[1])
+ cr.arc(self.start[0], self.start[1], theta_end_circle_size, 0,
+ 2.0 * numpy.pi)
+ cr.move_to(self.end[0] + theta_end_circle_size, self.end[1])
+ cr.arc(self.end[0], self.end[1], theta_end_circle_size, 0,
+ 2.0 * numpy.pi)
+ else:
+ start = get_xy(self.start)
+ control1 = get_xy(self.control1)
+ control2 = get_xy(self.control2)
+ end = get_xy(self.end)
+
+ draw_lines(cr, [
+ spline_eval(start, control1, control2, end, alpha)
+ for alpha in subdivide_spline(start, control1, control2, end)
+ ])
+
+ cr.move_to(start[0] + xy_end_circle_size, start[1])
+ cr.arc(start[0], start[1], xy_end_circle_size, 0, 2.0 * numpy.pi)
+ cr.move_to(end[0] + xy_end_circle_size, end[1])
+ cr.arc(end[0], end[1], xy_end_circle_size, 0, 2.0 * numpy.pi)
+
+ def ToThetaPoints(self):
+ t1, t2 = self.start
+ c_i_select = get_circular_index(self.start)
+ start = get_xy(self.start)
+ control1 = get_xy(self.control1)
+ control2 = get_xy(self.control2)
+ end = get_xy(self.end)
+
+ return [
+ to_theta_with_ci_and_derivs(
+ spline_eval(start, control1, control2, end, alpha - 0.00001),
+ spline_eval(start, control1, control2, end, alpha),
+ spline_eval(start, control1, control2, end, alpha + 0.00001),
+ c_i_select)
+ for alpha in subdivide_spline(start, control1, control2, end)
+ ]
+
+
+def get_derivs(t_prev, t, t_next):
+ c, a, b = t_prev, t, t_next
+ d1 = normalize(b - a)
+ d2 = normalize(c - a)
+ accel = (d1 + d2) / numpy.linalg.norm(a - b)
+ return (a[0], a[1], d1[0], d1[1], accel[0], accel[1])
+
+
+class ThetaSplineSegment:
+
+ def __init__(self,
+ start,
+ control1,
+ control2,
+ end,
+ name=None,
+ alpha_unitizer=None,
+ vmax=None):
+ self.start = start
+ self.control1 = control1
+ self.control2 = control2
+ self.end = end
+ self.name = name
+ self.alpha_unitizer = alpha_unitizer
+ self.vmax = vmax
+
+ def __repr__(self):
+ return "ThetaSplineSegment(%s, %s, &s, %s)" % (repr(
+ self.start), repr(self.control1), repr(
+ self.control2), repr(self.end))
+
+ def DrawTo(self, cr, theta_version):
+ if (theta_version):
+ draw_lines(cr, [
+ spline_eval(self.start, self.control1, self.control2, self.end,
+ alpha)
+ for alpha in subdivide_spline(self.start, self.control1,
+ self.control2, self.end)
+ ])
+ else:
+ start = get_xy(self.start)
+ end = get_xy(self.end)
+
+ draw_lines(cr, [
+ get_xy(
+ spline_eval(self.start, self.control1, self.control2,
+ self.end, alpha))
+ for alpha in subdivide_spline(self.start, self.control1,
+ self.control2, self.end)
+ ])
+
+ cr.move_to(start[0] + xy_end_circle_size, start[1])
+ cr.arc(start[0], start[1], xy_end_circle_size, 0, 2.0 * numpy.pi)
+ cr.move_to(end[0] + xy_end_circle_size, end[1])
+ cr.arc(end[0], end[1], xy_end_circle_size, 0, 2.0 * numpy.pi)
+
+ def ToThetaPoints(self):
+ return [
+ get_derivs(
+ spline_eval(self.start, self.control1, self.control2, self.end,
+ alpha - 0.00001),
+ spline_eval(self.start, self.control1, self.control2, self.end,
+ alpha),
+ spline_eval(self.start, self.control1, self.control2, self.end,
+ alpha + 0.00001))
+ for alpha in subdivide_spline(self.start, self.control1,
+ self.control2, self.end)
+ ]
+
+
+tall_box_x = 0.411
+tall_box_y = 0.125
+
+short_box_x = 0.431
+short_box_y = 0.082
+
+ready_above_box = to_theta_with_circular_index(tall_box_x,
+ tall_box_y + 0.08,
+ circular_index=-1)
+tall_box_grab = to_theta_with_circular_index(tall_box_x,
+ tall_box_y,
+ circular_index=-1)
+short_box_grab = to_theta_with_circular_index(short_box_x,
+ short_box_y,
+ circular_index=-1)
+
+# TODO(austin): Drive the front/back off the same numbers a bit better.
+front_high_box = to_theta_with_circular_index(0.378, 2.46, circular_index=-1)
+front_middle3_box = to_theta_with_circular_index(0.700,
+ 2.125,
+ circular_index=-1.000000)
+front_middle2_box = to_theta_with_circular_index(0.700,
+ 2.268,
+ circular_index=-1)
+front_middle1_box = to_theta_with_circular_index(0.800,
+ 1.915,
+ circular_index=-1)
+front_low_box = to_theta_with_circular_index(0.87, 1.572, circular_index=-1)
+back_high_box = to_theta_with_circular_index(-0.75, 2.48, circular_index=0)
+back_middle2_box = to_theta_with_circular_index(-0.700, 2.27, circular_index=0)
+back_middle1_box = to_theta_with_circular_index(-0.800, 1.93, circular_index=0)
+back_low_box = to_theta_with_circular_index(-0.87, 1.64, circular_index=0)
+
+back_extra_low_box = to_theta_with_circular_index(-0.87,
+ 1.52,
+ circular_index=0)
+
+front_switch = to_theta_with_circular_index(0.88, 0.967, circular_index=-1)
+back_switch = to_theta_with_circular_index(-0.88, 0.967, circular_index=-2)
+
+neutral = to_theta_with_circular_index(0.0, 0.33, circular_index=-1)
+
+up = to_theta_with_circular_index(0.0, 2.547, circular_index=-1)
+
+front_switch_auto = to_theta_with_circular_index(0.750,
+ 2.20,
+ circular_index=-1.000000)
+
+duck = numpy.array([numpy.pi / 2.0 - 0.92, numpy.pi / 2.0 - 4.26])
+
+starting = numpy.array([numpy.pi / 2.0 - 0.593329, numpy.pi / 2.0 - 3.749631])
+vertical_starting = numpy.array([numpy.pi / 2.0, -numpy.pi / 2.0])
+
+self_hang = numpy.array([numpy.pi / 2.0 - 0.191611, numpy.pi / 2.0])
+partner_hang = numpy.array([numpy.pi / 2.0 - (-0.30), numpy.pi / 2.0])
+
+above_hang = numpy.array([numpy.pi / 2.0 - 0.14, numpy.pi / 2.0 - (-0.165)])
+below_hang = numpy.array([numpy.pi / 2.0 - 0.39, numpy.pi / 2.0 - (-0.517)])
+
+up_c1 = to_theta((0.63, 1.17), circular_index=-1)
+up_c2 = to_theta((0.65, 1.62), circular_index=-1)
+
+front_high_box_c1 = to_theta((0.63, 1.04), circular_index=-1)
+front_high_box_c2 = to_theta((0.50, 1.60), circular_index=-1)
+
+front_middle2_box_c1 = to_theta((0.41, 0.83), circular_index=-1)
+front_middle2_box_c2 = to_theta((0.52, 1.30), circular_index=-1)
+
+front_middle1_box_c1 = to_theta((0.34, 0.82), circular_index=-1)
+front_middle1_box_c2 = to_theta((0.48, 1.15), circular_index=-1)
+
+#c1: (1.421433, -1.070254)
+#c2: (1.434384, -1.057803
+ready_above_box_c1 = numpy.array([1.480802, -1.081218])
+ready_above_box_c2 = numpy.array([1.391449, -1.060331])
+
+front_switch_c1 = numpy.array([1.903841, -0.622351])
+front_switch_c2 = numpy.array([1.903841, -0.622351])
+
+
+sparse_front_points = [
+ (front_high_box, "FrontHighBox"),
+ (front_middle2_box, "FrontMiddle2Box"),
+ (front_middle3_box, "FrontMiddle3Box"),
+ (front_middle1_box, "FrontMiddle1Box"),
+ (front_low_box, "FrontLowBox"),
+ (front_switch, "FrontSwitch"),
+] # yapf: disable
+
+sparse_back_points = [
+ (back_high_box, "BackHighBox"),
+ (back_middle2_box, "BackMiddle2Box"),
+ (back_middle1_box, "BackMiddle1Box"),
+ (back_low_box, "BackLowBox"),
+ (back_extra_low_box, "BackExtraLowBox"),
+] # yapf: disable
+
+def expand_points(points, max_distance):
+ """Expands a list of points to be at most max_distance apart
+
+ Generates the paths to connect the new points to the closest input points,
+ and the paths connecting the points.
+
+ Args:
+ points, list of tuple of point, name, The points to start with and fill
+ in.
+ max_distance, float, The max distance between two points when expanding
+ the graph.
+
+ Return:
+ points, edges
+ """
+ result_points = [points[0]]
+ result_paths = []
+ for point, name in points[1:]:
+ previous_point = result_points[-1][0]
+ previous_point_xy = get_xy(previous_point)
+ circular_index = get_circular_index(previous_point)
+
+ point_xy = get_xy(point)
+ norm = numpy.linalg.norm(point_xy - previous_point_xy)
+ num_points = int(numpy.ceil(norm / max_distance))
+ last_iteration_point = previous_point
+ for subindex in range(1, num_points):
+ subpoint = to_theta(alpha_blend(previous_point_xy, point_xy,
+ float(subindex) / num_points),
+ circular_index=circular_index)
+ result_points.append(
+ (subpoint, '%s%dof%d' % (name, subindex, num_points)))
+ result_paths.append(
+ XYSegment(last_iteration_point, subpoint, vmax=6.0))
+ if (last_iteration_point != previous_point).any():
+ result_paths.append(XYSegment(previous_point, subpoint))
+ if subindex == num_points - 1:
+ result_paths.append(XYSegment(subpoint, point, vmax=6.0))
+ else:
+ result_paths.append(XYSegment(subpoint, point))
+ last_iteration_point = subpoint
+ result_points.append((point, name))
+
+ return result_points, result_paths
+
+
+front_points, front_paths = expand_points(sparse_front_points, 0.06)
+back_points, back_paths = expand_points(sparse_back_points, 0.06)
+
+points = [(ready_above_box, "ReadyAboveBox"),
+ (tall_box_grab, "TallBoxGrab"),
+ (short_box_grab, "ShortBoxGrab"),
+ (back_switch, "BackSwitch"),
+ (neutral, "Neutral"),
+ (up, "Up"),
+ (above_hang, "AboveHang"),
+ (below_hang, "BelowHang"),
+ (self_hang, "SelfHang"),
+ (partner_hang, "PartnerHang"),
+ (front_switch_auto, "FrontSwitchAuto"),
+ (starting, "Starting"),
+ (duck, "Duck"),
+ (vertical_starting, "VerticalStarting"),
+] + front_points + back_points # yapf: disable
+
+duck_c1 = numpy.array([1.337111, -1.721008])
+duck_c2 = numpy.array([1.283701, -1.795519])
+
+ready_to_up_c1 = numpy.array([1.792962, 0.198329])
+ready_to_up_c2 = numpy.array([1.792962, 0.198329])
+
+front_switch_auto_c1 = numpy.array([1.792857, -0.372768])
+front_switch_auto_c2 = numpy.array([1.861885, -0.273664])
+
+# We need to define critical points so we can create paths connecting them.
+# TODO(austin): Attach velocities to the slow ones.
+ready_to_back_low_c1 = numpy.array([2.524325, 0.046417])
+
+neutral_to_back_low_c1 = numpy.array([2.381942, -0.070220])
+
+tall_to_back_low_c1 = numpy.array([2.603918, 0.088298])
+tall_to_back_low_c2 = numpy.array([1.605624, 1.003434])
+
+tall_to_back_high_c2 = numpy.array([1.508610, 0.946147])
+
+# If true, only plot the first named segment
+isolate = False
+
+long_alpha_unitizer = numpy.matrix([[1.0 / 17.0, 0.0], [0.0, 1.0 / 17.0]])
+
+neutral_to_back_c1 = numpy.array([0.702527, -2.618276])
+neutral_to_back_c2 = numpy.array([0.526914, -3.109691])
+
+named_segments = [
+ ThetaSplineSegment(neutral, neutral_to_back_c1, neutral_to_back_c2,
+ back_switch, "BackSwitch"),
+ ThetaSplineSegment(neutral,
+ neutral_to_back_low_c1,
+ tall_to_back_high_c2,
+ back_high_box,
+ "NeutralBoxToHigh",
+ alpha_unitizer=long_alpha_unitizer),
+ ThetaSplineSegment(neutral, neutral_to_back_low_c1, tall_to_back_high_c2,
+ back_middle2_box, "NeutralBoxToMiddle2",
+ long_alpha_unitizer),
+ ThetaSplineSegment(neutral, neutral_to_back_low_c1, tall_to_back_low_c2,
+ back_middle1_box, "NeutralBoxToMiddle1",
+ long_alpha_unitizer),
+ ThetaSplineSegment(neutral, neutral_to_back_low_c1, tall_to_back_low_c2,
+ back_low_box, "NeutralBoxToLow", long_alpha_unitizer),
+ ThetaSplineSegment(ready_above_box, ready_to_back_low_c1,
+ tall_to_back_high_c2, back_high_box, "ReadyBoxToHigh",
+ long_alpha_unitizer),
+ ThetaSplineSegment(ready_above_box, ready_to_back_low_c1,
+ tall_to_back_high_c2, back_middle2_box,
+ "ReadyBoxToMiddle2", long_alpha_unitizer),
+ ThetaSplineSegment(ready_above_box, ready_to_back_low_c1,
+ tall_to_back_low_c2, back_middle1_box,
+ "ReadyBoxToMiddle1", long_alpha_unitizer),
+ ThetaSplineSegment(ready_above_box, ready_to_back_low_c1,
+ tall_to_back_low_c2, back_low_box, "ReadyBoxToLow",
+ long_alpha_unitizer),
+ ThetaSplineSegment(short_box_grab, tall_to_back_low_c1,
+ tall_to_back_high_c2, back_high_box, "ShortBoxToHigh",
+ long_alpha_unitizer),
+ ThetaSplineSegment(short_box_grab, tall_to_back_low_c1,
+ tall_to_back_high_c2, back_middle2_box,
+ "ShortBoxToMiddle2", long_alpha_unitizer),
+ ThetaSplineSegment(short_box_grab, tall_to_back_low_c1,
+ tall_to_back_low_c2, back_middle1_box,
+ "ShortBoxToMiddle1", long_alpha_unitizer),
+ ThetaSplineSegment(short_box_grab, tall_to_back_low_c1,
+ tall_to_back_low_c2, back_low_box, "ShortBoxToLow",
+ long_alpha_unitizer),
+ ThetaSplineSegment(tall_box_grab, tall_to_back_low_c1,
+ tall_to_back_high_c2, back_high_box, "TallBoxToHigh",
+ long_alpha_unitizer),
+ ThetaSplineSegment(tall_box_grab, tall_to_back_low_c1,
+ tall_to_back_high_c2, back_middle2_box,
+ "TallBoxToMiddle2", long_alpha_unitizer),
+ ThetaSplineSegment(tall_box_grab, tall_to_back_low_c1, tall_to_back_low_c2,
+ back_middle1_box, "TallBoxToMiddle1",
+ long_alpha_unitizer),
+ ThetaSplineSegment(tall_box_grab, tall_to_back_low_c1, tall_to_back_low_c2,
+ back_low_box, "TallBoxToLow", long_alpha_unitizer),
+ SplineSegment(neutral, ready_above_box_c1, ready_above_box_c2,
+ ready_above_box, "ReadyToNeutral"),
+ XYSegment(ready_above_box, tall_box_grab, "ReadyToTallBox", vmax=6.0),
+ XYSegment(ready_above_box, short_box_grab, "ReadyToShortBox", vmax=6.0),
+ XYSegment(tall_box_grab, short_box_grab, "TallToShortBox", vmax=6.0),
+ SplineSegment(neutral, ready_above_box_c1, ready_above_box_c2,
+ tall_box_grab, "TallToNeutral"),
+ SplineSegment(neutral, ready_above_box_c1, ready_above_box_c2,
+ short_box_grab, "ShortToNeutral"),
+ SplineSegment(neutral, up_c1, up_c2, up, "NeutralToUp"),
+ SplineSegment(neutral, front_high_box_c1, front_high_box_c2,
+ front_high_box, "NeutralToFrontHigh"),
+ SplineSegment(neutral, front_middle2_box_c1, front_middle2_box_c2,
+ front_middle2_box, "NeutralToFrontMiddle2"),
+ SplineSegment(neutral, front_middle1_box_c1, front_middle1_box_c2,
+ front_middle1_box, "NeutralToFrontMiddle1"),
+]
+
+unnamed_segments = [
+ SplineSegment(neutral, front_switch_auto_c1, front_switch_auto_c2,
+ front_switch_auto),
+ SplineSegment(tall_box_grab, ready_to_up_c1, ready_to_up_c2, up),
+ SplineSegment(short_box_grab, ready_to_up_c1, ready_to_up_c2, up),
+ SplineSegment(ready_above_box, ready_to_up_c1, ready_to_up_c2, up),
+ ThetaSplineSegment(duck, duck_c1, duck_c2, neutral),
+ SplineSegment(neutral, front_switch_c1, front_switch_c2, front_switch),
+ XYSegment(ready_above_box, front_low_box),
+ XYSegment(ready_above_box, front_switch),
+ XYSegment(ready_above_box, front_middle1_box),
+ XYSegment(ready_above_box, front_middle2_box),
+ XYSegment(ready_above_box, front_middle3_box),
+ SplineSegment(ready_above_box, ready_to_up_c1, ready_to_up_c2,
+ front_high_box),
+ AngleSegment(starting, vertical_starting),
+ AngleSegment(vertical_starting, neutral),
+ XYSegment(neutral, front_low_box),
+ XYSegment(up, front_high_box),
+ XYSegment(up, front_middle2_box),
+ XYSegment(front_middle3_box, up),
+ XYSegment(front_middle3_box, front_high_box),
+ XYSegment(front_middle3_box, front_middle2_box),
+ XYSegment(front_middle3_box, front_middle1_box),
+ XYSegment(up, front_middle1_box),
+ XYSegment(up, front_low_box),
+ XYSegment(front_high_box, front_middle2_box),
+ XYSegment(front_high_box, front_middle1_box),
+ XYSegment(front_high_box, front_low_box),
+ XYSegment(front_middle2_box, front_middle1_box),
+ XYSegment(front_middle2_box, front_low_box),
+ XYSegment(front_middle1_box, front_low_box),
+ XYSegment(front_switch, front_low_box),
+ XYSegment(front_switch, up),
+ XYSegment(front_switch, front_high_box),
+ AngleSegment(up, back_high_box),
+ AngleSegment(up, back_middle2_box),
+ AngleSegment(up, back_middle1_box),
+ AngleSegment(up, back_low_box),
+ XYSegment(back_high_box, back_middle2_box),
+ XYSegment(back_high_box, back_middle1_box),
+ XYSegment(back_high_box, back_low_box),
+ XYSegment(back_middle2_box, back_middle1_box),
+ XYSegment(back_middle2_box, back_low_box),
+ XYSegment(back_middle1_box, back_low_box),
+ AngleSegment(up, above_hang),
+ AngleSegment(above_hang, below_hang),
+ AngleSegment(up, below_hang),
+ AngleSegment(up, self_hang),
+ AngleSegment(up, partner_hang),
+] + front_paths + back_paths
+
+segments = []
+if isolate:
+ segments += named_segments[:isolate]
+else:
+ segments += named_segments + unnamed_segments
diff --git a/y2023/y2023.json b/y2023/y2023.json
index 76f0e52..d5f9462 100644
--- a/y2023/y2023.json
+++ b/y2023/y2023.json
@@ -1,5 +1,5 @@
{
- "channel_storage_duration": 2000000000,
+ "channel_storage_duration": 10000000000,
"maps": [
{
"match": {
diff --git a/y2023/y2023_pi_template.json b/y2023/y2023_pi_template.json
index 592aa14..fac37b2 100644
--- a/y2023/y2023_pi_template.json
+++ b/y2023/y2023_pi_template.json
@@ -164,7 +164,7 @@
"type": "frc971.vision.CameraImage",
"source_node": "pi{{ NUM }}",
"frequency": 40,
- "max_size": 4200000,
+ "max_size": 1843456,
"num_readers": 4,
"read_method": "PIN",
"num_senders": 18