52
52
import org .apache .iceberg .exceptions .RuntimeIOException ;
53
53
import org .apache .iceberg .exceptions .ValidationException ;
54
54
import org .apache .iceberg .io .FileIO ;
55
+ import org .apache .iceberg .relocated .com .google .common .base .Preconditions ;
55
56
import org .apache .iceberg .util .LocationUtil ;
56
57
import org .slf4j .Logger ;
57
58
import org .slf4j .LoggerFactory ;
@@ -71,6 +72,8 @@ public class EtcdCatalog extends BaseMetastoreCatalog implements SupportsNamespa
71
72
72
73
public EtcdCatalog (String name , String uri , String warehouseLocation , FileIO io ) {
73
74
this .catalogName = name ;
75
+ Preconditions .checkArgument (
76
+ name != null && !name .isBlank () && !name .contains ("/" ), "Invalid catalog name" );
74
77
this .warehouseLocation = LocationUtil .stripTrailingSlash (warehouseLocation );
75
78
var etcdClient =
76
79
Client .builder ().endpoints (uri .split ("," )).keepaliveWithoutCalls (false ).build ();
@@ -124,8 +127,15 @@ public boolean namespaceExists(Namespace namespace) {
124
127
return unwrap (kv .get (key , GetOption .builder ().withCountOnly (true ).build ())).getCount () > 0 ;
125
128
}
126
129
127
- private static String namespaceKey (Namespace namespace ) {
128
- return NAMESPACE_PREFIX + namespaceToPath (namespace );
130
+ private String namespaceKey (Namespace namespace ) {
131
+ return namespacePrefix () + namespaceToPath (namespace );
132
+ }
133
+
134
+ private String namespacePrefix () {
135
+ if ("default" .equals (catalogName )) { // for backward-compatibility
136
+ return NAMESPACE_PREFIX ;
137
+ }
138
+ return catalogName + "/" + NAMESPACE_PREFIX ;
129
139
}
130
140
131
141
private static <T > T unwrapCommit (java .util .concurrent .CompletableFuture <T > x ) {
@@ -195,9 +205,7 @@ private void validateTableIdentifier(TableIdentifier identifier) {
195
205
public List <Namespace > listNamespaces (Namespace namespace ) throws NoSuchNamespaceException {
196
206
validateNamespace (namespace );
197
207
String prefix = namespaceKey (namespace );
198
- if (!namespace .isEmpty ()) {
199
- prefix += "/" ;
200
- }
208
+ prefix = prefix .endsWith ("/" ) ? prefix : prefix + "/" ;
201
209
GetResponse res = unwrap (kv .get (byteSeq (prefix ), GetOption .builder ().isPrefix (true ).build ()));
202
210
if (res .getKvs ().isEmpty () && !namespace .isEmpty ()) {
203
211
if (!namespaceExists (namespace )) {
@@ -209,7 +217,7 @@ public List<Namespace> listNamespaces(Namespace namespace) throws NoSuchNamespac
209
217
k ->
210
218
Namespace .of (
211
219
Strings .removePrefix (
212
- k .getKey ().toString (StandardCharsets .UTF_8 ), NAMESPACE_PREFIX )
220
+ k .getKey ().toString (StandardCharsets .UTF_8 ), namespacePrefix () )
213
221
.split ("/" )))
214
222
.toList ();
215
223
}
@@ -330,9 +338,7 @@ protected TableOperations newTableOps(TableIdentifier tableIdentifier) {
330
338
public List <TableIdentifier > listTables (Namespace namespace ) {
331
339
validateNamespace (namespace );
332
340
String prefix = tableKey (namespace );
333
- if (!namespace .isEmpty ()) {
334
- prefix += "/" ;
335
- }
341
+ prefix = prefix .endsWith ("/" ) ? prefix : prefix + "/" ;
336
342
GetResponse res = unwrap (kv .get (byteSeq (prefix ), GetOption .builder ().isPrefix (true ).build ()));
337
343
if (res .getKvs ().isEmpty () && !namespace .isEmpty ()) {
338
344
if (!namespaceExists (namespace )) {
@@ -343,17 +349,24 @@ public List<TableIdentifier> listTables(Namespace namespace) {
343
349
.map (
344
350
k ->
345
351
TableIdentifier .of (
346
- Strings .removePrefix (k .getKey ().toString (StandardCharsets .UTF_8 ), TABLE_PREFIX )
352
+ Strings .removePrefix (k .getKey ().toString (StandardCharsets .UTF_8 ), tablePrefix () )
347
353
.split ("/" )))
348
354
.toList ();
349
355
}
350
356
351
- private static String tableKey (Namespace namespace ) {
352
- return TABLE_PREFIX + namespaceToPath (namespace );
357
+ private String tableKey (Namespace namespace ) {
358
+ return tablePrefix () + namespaceToPath (namespace );
359
+ }
360
+
361
+ private String tableKey (TableIdentifier identifier ) {
362
+ return tablePrefix () + tableIdentifierToPath (identifier );
353
363
}
354
364
355
- private static String tableKey (TableIdentifier identifier ) {
356
- return TABLE_PREFIX + tableIdentifierToPath (identifier );
365
+ private String tablePrefix () {
366
+ if ("default" .equals (catalogName )) { // for backward-compatibility
367
+ return TABLE_PREFIX ;
368
+ }
369
+ return catalogName + "/" + TABLE_PREFIX ;
357
370
}
358
371
359
372
private static String tableIdentifierToPath (TableIdentifier identifier ) {
0 commit comments