feat: Add Iceberg REST Catalog server and admin UI (#8175)
* feat: Add Iceberg REST Catalog server Implement Iceberg REST Catalog API on a separate port (default 8181) that exposes S3 Tables metadata through the Apache Iceberg REST protocol. - Add new weed/s3api/iceberg package with REST handlers - Implement /v1/config endpoint returning catalog configuration - Implement namespace endpoints (list/create/get/head/delete) - Implement table endpoints (list/create/load/head/delete/update) - Add -port.iceberg flag to S3 standalone server (s3.go) - Add -s3.port.iceberg flag to combined server mode (server.go) - Add -s3.port.iceberg flag to mini cluster mode (mini.go) - Support prefix-based routing for multiple catalogs The Iceberg REST server reuses S3 Tables metadata storage under /table-buckets and enables DuckDB, Spark, and other Iceberg clients to connect to SeaweedFS as a catalog. * feat: Add Iceberg Catalog pages to admin UI Add admin UI pages to browse Iceberg catalogs, namespaces, and tables. - Add Iceberg Catalog menu item under Object Store navigation - Create iceberg_catalog.templ showing catalog overview with REST info - Create iceberg_namespaces.templ listing namespaces in a catalog - Create iceberg_tables.templ listing tables in a namespace - Add handlers and routes in admin_handlers.go - Add Iceberg data provider methods in s3tables_management.go - Add Iceberg data types in types.go The Iceberg Catalog pages provide visibility into the same S3 Tables data through an Iceberg-centric lens, including REST endpoint examples for DuckDB and PyIceberg. * test: Add Iceberg catalog integration tests and reorg s3tables tests - Reorganize existing s3tables tests to test/s3tables/table-buckets/ - Add new test/s3tables/catalog/ for Iceberg REST catalog tests - Add TestIcebergConfig to verify /v1/config endpoint - Add TestIcebergNamespaces to verify namespace listing - Add TestDuckDBIntegration for DuckDB connectivity (requires Docker) - Update CI workflow to use new test paths * fix: Generate proper random UUIDs for Iceberg tables Address code review feedback: - Replace placeholder UUID with crypto/rand-based UUID v4 generation - Add detailed TODO comments for handleUpdateTable stub explaining the required atomic metadata swap implementation * fix: Serve Iceberg on localhost listener when binding to different interface Address code review feedback: properly serve the localhost listener when the Iceberg server is bound to a non-localhost interface. * ci: Add Iceberg catalog integration tests to CI Add new job to run Iceberg catalog tests in CI, along with: - Iceberg package build verification - Iceberg unit tests - Iceberg go vet checks - Iceberg format checks * fix: Address code review feedback for Iceberg implementation - fix: Replace hardcoded account ID with s3_constants.AccountAdminId in buildTableBucketARN() - fix: Improve UUID generation error handling with deterministic fallback (timestamp + PID + counter) - fix: Update handleUpdateTable to return HTTP 501 Not Implemented instead of fake success - fix: Better error handling in handleNamespaceExists to distinguish 404 from 500 errors - fix: Use relative URL in template instead of hardcoded localhost:8181 - fix: Add HTTP timeout to test's waitForService function to avoid hangs - fix: Use dynamic ephemeral ports in integration tests to avoid flaky parallel failures - fix: Add Iceberg port to final port configuration logging in mini.go * fix: Address critical issues in Iceberg implementation - fix: Cache table UUIDs to ensure persistence across LoadTable calls The UUID now remains stable for the lifetime of the server session. TODO: For production, UUIDs should be persisted in S3 Tables metadata. - fix: Remove redundant URL-encoded namespace parsing mux router already decodes %1F to \x1F before passing to handlers. Redundant ReplaceAll call could cause bugs with literal %1F in namespace. * fix: Improve test robustness and reduce code duplication - fix: Make DuckDB test more robust by failing on unexpected errors Instead of silently logging errors, now explicitly check for expected conditions (extension not available) and skip the test appropriately. - fix: Extract username helper method to reduce duplication Created getUsername() helper in AdminHandlers to avoid duplicating the username retrieval logic across Iceberg page handlers. * fix: Add mutex protection to table UUID cache Protects concurrent access to the tableUUIDs map with sync.RWMutex. Uses read-lock for fast path when UUID already cached, and write-lock for generating new UUIDs. Includes double-check pattern to handle race condition between read-unlock and write-lock. * style: fix go fmt errors * feat(iceberg): persist table UUID in S3 Tables metadata * feat(admin): configure Iceberg port in Admin UI and commands * refactor: address review comments (flags, tests, handlers) - command/mini: fix tracking of explicit s3.port.iceberg flag - command/admin: add explicit -iceberg.port flag - admin/handlers: reuse getUsername helper - tests: use 127.0.0.1 for ephemeral ports and os.Stat for file size check * test: check error from FileStat in verify_gc_empty_test
This commit is contained in:
@@ -91,6 +91,9 @@ func (h *AdminHandlers) SetupRoutes(r *gin.Engine, authRequired bool, adminUser,
|
||||
protected.GET("/object-store/s3tables/buckets", h.ShowS3TablesBuckets)
|
||||
protected.GET("/object-store/s3tables/buckets/:bucket/namespaces", h.ShowS3TablesNamespaces)
|
||||
protected.GET("/object-store/s3tables/buckets/:bucket/namespaces/:namespace/tables", h.ShowS3TablesTables)
|
||||
protected.GET("/object-store/iceberg", h.ShowIcebergCatalog)
|
||||
protected.GET("/object-store/iceberg/:catalog/namespaces", h.ShowIcebergNamespaces)
|
||||
protected.GET("/object-store/iceberg/:catalog/namespaces/:namespace/tables", h.ShowIcebergTables)
|
||||
|
||||
// File browser routes
|
||||
protected.GET("/files", h.fileBrowserHandlers.ShowFileBrowser)
|
||||
@@ -259,6 +262,9 @@ func (h *AdminHandlers) SetupRoutes(r *gin.Engine, authRequired bool, adminUser,
|
||||
r.GET("/object-store/s3tables/buckets", h.ShowS3TablesBuckets)
|
||||
r.GET("/object-store/s3tables/buckets/:bucket/namespaces", h.ShowS3TablesNamespaces)
|
||||
r.GET("/object-store/s3tables/buckets/:bucket/namespaces/:namespace/tables", h.ShowS3TablesTables)
|
||||
r.GET("/object-store/iceberg", h.ShowIcebergCatalog)
|
||||
r.GET("/object-store/iceberg/:catalog/namespaces", h.ShowIcebergNamespaces)
|
||||
r.GET("/object-store/iceberg/:catalog/namespaces/:namespace/tables", h.ShowIcebergTables)
|
||||
|
||||
// File browser routes
|
||||
r.GET("/files", h.fileBrowserHandlers.ShowFileBrowser)
|
||||
@@ -454,10 +460,7 @@ func (h *AdminHandlers) ShowS3Buckets(c *gin.Context) {
|
||||
|
||||
// ShowS3TablesBuckets renders the S3 Tables buckets page
|
||||
func (h *AdminHandlers) ShowS3TablesBuckets(c *gin.Context) {
|
||||
username := c.GetString("username")
|
||||
if username == "" {
|
||||
username = "admin"
|
||||
}
|
||||
username := h.getUsername(c)
|
||||
|
||||
data, err := h.adminServer.GetS3TablesBucketsData(c.Request.Context())
|
||||
if err != nil {
|
||||
@@ -476,10 +479,7 @@ func (h *AdminHandlers) ShowS3TablesBuckets(c *gin.Context) {
|
||||
|
||||
// ShowS3TablesNamespaces renders namespaces for a table bucket
|
||||
func (h *AdminHandlers) ShowS3TablesNamespaces(c *gin.Context) {
|
||||
username := c.GetString("username")
|
||||
if username == "" {
|
||||
username = "admin"
|
||||
}
|
||||
username := h.getUsername(c)
|
||||
|
||||
bucketName := c.Param("bucket")
|
||||
arn, err := buildS3TablesBucketArn(bucketName)
|
||||
@@ -505,10 +505,7 @@ func (h *AdminHandlers) ShowS3TablesNamespaces(c *gin.Context) {
|
||||
|
||||
// ShowS3TablesTables renders tables for a namespace
|
||||
func (h *AdminHandlers) ShowS3TablesTables(c *gin.Context) {
|
||||
username := c.GetString("username")
|
||||
if username == "" {
|
||||
username = "admin"
|
||||
}
|
||||
username := h.getUsername(c)
|
||||
|
||||
bucketName := c.Param("bucket")
|
||||
namespace := c.Param("namespace")
|
||||
@@ -537,6 +534,81 @@ func buildS3TablesBucketArn(bucketName string) (string, error) {
|
||||
return s3tables.BuildBucketARN(s3tables.DefaultRegion, s3_constants.AccountAdminId, bucketName)
|
||||
}
|
||||
|
||||
// getUsername returns the username from context, defaulting to "admin" if not set
|
||||
func (h *AdminHandlers) getUsername(c *gin.Context) string {
|
||||
username := c.GetString("username")
|
||||
if username == "" {
|
||||
username = "admin"
|
||||
}
|
||||
return username
|
||||
}
|
||||
|
||||
// ShowIcebergCatalog renders the Iceberg Catalog overview page
|
||||
func (h *AdminHandlers) ShowIcebergCatalog(c *gin.Context) {
|
||||
data, err := h.adminServer.GetIcebergCatalogData(c.Request.Context())
|
||||
if err != nil {
|
||||
c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to get Iceberg catalog data: " + err.Error()})
|
||||
return
|
||||
}
|
||||
data.Username = h.getUsername(c)
|
||||
|
||||
c.Header("Content-Type", "text/html")
|
||||
component := app.IcebergCatalog(data)
|
||||
layoutComponent := layout.Layout(c, component)
|
||||
if err := layoutComponent.Render(c.Request.Context(), c.Writer); err != nil {
|
||||
c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to render template: " + err.Error()})
|
||||
}
|
||||
}
|
||||
|
||||
// ShowIcebergNamespaces renders namespaces for an Iceberg catalog
|
||||
func (h *AdminHandlers) ShowIcebergNamespaces(c *gin.Context) {
|
||||
catalogName := c.Param("catalog")
|
||||
arn, err := buildS3TablesBucketArn(catalogName)
|
||||
if err != nil {
|
||||
c.JSON(http.StatusBadRequest, gin.H{"error": err.Error()})
|
||||
return
|
||||
}
|
||||
|
||||
data, err := h.adminServer.GetIcebergNamespacesData(c.Request.Context(), catalogName, arn)
|
||||
if err != nil {
|
||||
c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to get Iceberg namespaces: " + err.Error()})
|
||||
return
|
||||
}
|
||||
data.Username = h.getUsername(c)
|
||||
|
||||
c.Header("Content-Type", "text/html")
|
||||
component := app.IcebergNamespaces(data)
|
||||
layoutComponent := layout.Layout(c, component)
|
||||
if err := layoutComponent.Render(c.Request.Context(), c.Writer); err != nil {
|
||||
c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to render template: " + err.Error()})
|
||||
}
|
||||
}
|
||||
|
||||
// ShowIcebergTables renders tables for an Iceberg namespace
|
||||
func (h *AdminHandlers) ShowIcebergTables(c *gin.Context) {
|
||||
catalogName := c.Param("catalog")
|
||||
namespace := c.Param("namespace")
|
||||
arn, err := buildS3TablesBucketArn(catalogName)
|
||||
if err != nil {
|
||||
c.JSON(http.StatusBadRequest, gin.H{"error": err.Error()})
|
||||
return
|
||||
}
|
||||
|
||||
data, err := h.adminServer.GetIcebergTablesData(c.Request.Context(), catalogName, arn, namespace)
|
||||
if err != nil {
|
||||
c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to get Iceberg tables: " + err.Error()})
|
||||
return
|
||||
}
|
||||
data.Username = h.getUsername(c)
|
||||
|
||||
c.Header("Content-Type", "text/html")
|
||||
component := app.IcebergTables(data)
|
||||
layoutComponent := layout.Layout(c, component)
|
||||
if err := layoutComponent.Render(c.Request.Context(), c.Writer); err != nil {
|
||||
c.JSON(http.StatusInternalServerError, gin.H{"error": "Failed to render template: " + err.Error()})
|
||||
}
|
||||
}
|
||||
|
||||
// ShowBucketDetails returns detailed information about a specific bucket
|
||||
func (h *AdminHandlers) ShowBucketDetails(c *gin.Context) {
|
||||
bucketName := c.Param("bucket")
|
||||
|
||||
Reference in New Issue
Block a user